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 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 sig 3 E80B8FFD 2012-03-05 P. Taylor Goetz -----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 " 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 ================================================ %-4r [%t] %-5p %c - %m%n ================================================ 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 ================================================ ${storm.home}/logs/${logfile.name} ${storm.home}/logs/${logfile.name}.%i 1 9 100MB %d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n ${storm.home}/logs/access.log ${storm.home}/logs/access.log.%i 1 9 100MB %d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n ${storm.home}/logs/metrics.log metrics.log.%i 1 9 2MB %d %-8r %m%n ================================================ 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* (:batch-transfer-queue executor-data) (disruptor/handler [o seq-id batch-end?] (let [^ArrayList alist (.getObject cached-emit)] (.add alist o) (when batch-end? (worker-transfer-fn serializer alist) (.setObject cached-emit (ArrayList.)) ))) :kill-fn (:report-error-and-die executor-data)))) (defn setup-metrics! [executor-data] (let [{:keys [storm-conf receive-queue worker-context interval->task->metric-registry]} executor-data distinct-time-bucket-intervals (keys interval->task->metric-registry)] (doseq [interval distinct-time-bucket-intervals] (schedule-recurring (:user-timer (:worker executor-data)) interval interval (fn [] (disruptor/publish receive-queue [[nil (TupleImpl. worker-context [interval] Constants/SYSTEM_TASK_ID Constants/METRICS_TICK_STREAM_ID)]])))))) (defn metrics-tick [executor-data task-datas ^TupleImpl tuple] (let [{:keys [interval->task->metric-registry ^WorkerTopologyContext worker-context]} executor-data interval (.getInteger tuple 0)] (doseq [[task-id task-data] task-datas :let [name->imetric (-> interval->task->metric-registry (get interval) (get task-id)) task-info (IMetricsConsumer$TaskInfo. (. (java.net.InetAddress/getLocalHost) getCanonicalHostName) (.getThisWorkerPort worker-context) (:component-id executor-data) task-id (long (/ (System/currentTimeMillis) 1000)) interval) data-points (->> name->imetric (map (fn [[name imetric]] (let [value (.getValueAndReset ^IMetric imetric)] (if value (IMetricsConsumer$DataPoint. name value))))) (filter identity) (into []))]] (if (seq data-points) (task/send-unanchored task-data Constants/METRICS_STREAM_ID [task-info data-points]))))) (defn setup-ticks! [worker executor-data] (let [storm-conf (:storm-conf executor-data) tick-time-secs (storm-conf TOPOLOGY-TICK-TUPLE-FREQ-SECS) receive-queue (:receive-queue executor-data) context (:worker-context executor-data)] (when tick-time-secs (if (or (system-id? (:component-id executor-data)) (and (not (storm-conf TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS)) (= :spout (:type executor-data)))) (log-message "Timeouts disabled for executor " (:component-id executor-data) ":" (:executor-id executor-data)) (schedule-recurring (:user-timer worker) tick-time-secs tick-time-secs (fn [] (disruptor/publish receive-queue [[nil (TupleImpl. context [tick-time-secs] Constants/SYSTEM_TASK_ID Constants/SYSTEM_TICK_STREAM_ID)]] ))))))) (defn mk-executor [worker executor-id] (let [executor-data (mk-executor-data worker executor-id) _ (log-message "Loading executor " (:component-id executor-data) ":" (pr-str executor-id)) task-datas (->> executor-data :task-ids (map (fn [t] [t (task/mk-task executor-data t)])) (into {}) (HashMap.)) _ (log-message "Loaded executor tasks " (:component-id executor-data) ":" (pr-str executor-id)) report-error-and-die (:report-error-and-die executor-data) component-id (:component-id executor-data) ;; starting the batch-transfer->worker ensures that anything publishing to that queue ;; doesn't block (because it's a single threaded queue and the caching/consumer started ;; trick isn't thread-safe) system-threads [(start-batch-transfer->worker-handler! worker executor-data)] handlers (with-error-reaction report-error-and-die (mk-threads executor-data task-datas)) threads (concat handlers system-threads)] (setup-ticks! worker executor-data) (log-message "Finished loading executor " component-id ":" (pr-str executor-id)) ;; TODO: add method here to get rendered stats... have worker call that when heartbeating (reify RunningExecutor (render-stats [this] (stats/render-stats! (:stats executor-data))) (get-executor-id [this] executor-id ) Shutdownable (shutdown [this] (log-message "Shutting down executor " component-id ":" (pr-str executor-id)) (disruptor/halt-with-interrupt! (:receive-queue executor-data)) (disruptor/halt-with-interrupt! (:batch-transfer-queue executor-data)) (doseq [t threads] (.interrupt t) (.join t)) (doseq [user-context (map :user-context (vals task-datas))] (doseq [hook (.getHooks user-context)] (.cleanup hook))) (.disconnect (:storm-cluster-state executor-data)) (when @(:open-or-prepare-was-called? executor-data) (doseq [obj (map :object (vals task-datas))] (close-component executor-data obj))) (log-message "Shut down executor " component-id ":" (pr-str executor-id))) ))) (defn- fail-spout-msg [executor-data task-data msg-id tuple-info time-delta] (let [^ISpout spout (:object task-data) task-id (:task-id task-data)] ;;TODO: need to throttle these when there's lots of failures (log-debug "Failing message " msg-id ": " tuple-info) (.fail spout msg-id) (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta)) (when time-delta (builtin-metrics/spout-failed-tuple! (:builtin-metrics task-data) (:stats executor-data) (:stream tuple-info)) (stats/spout-failed-tuple! (:stats executor-data) (:stream tuple-info) time-delta)))) (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta] (let [storm-conf (:storm-conf executor-data) ^ISpout spout (:object task-data) task-id (:task-id task-data)] (when (= true (storm-conf TOPOLOGY-DEBUG)) (log-message "Acking message " msg-id)) (.ack spout msg-id) (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta)) (when time-delta (builtin-metrics/spout-acked-tuple! (:builtin-metrics task-data) (:stats executor-data) (:stream tuple-info) time-delta) (stats/spout-acked-tuple! (:stats executor-data) (:stream tuple-info) time-delta)))) (defn mk-task-receiver [executor-data tuple-action-fn] (let [^KryoTupleDeserializer deserializer (:deserializer executor-data) task-ids (:task-ids executor-data) debug? (= true (-> executor-data :storm-conf (get TOPOLOGY-DEBUG))) ] (disruptor/clojure-handler (fn [tuple-batch sequence-id end-of-batch?] (fast-list-iter [[task-id msg] tuple-batch] (let [^TupleImpl tuple (if (instance? Tuple msg) msg (.deserialize deserializer msg))] (when debug? (log-message "Processing received message " tuple)) (if task-id (tuple-action-fn task-id tuple) ;; null task ids are broadcast tuples (fast-list-iter [task-id task-ids] (tuple-action-fn task-id tuple) )) )))))) (defn executor-max-spout-pending [storm-conf num-tasks] (let [p (storm-conf TOPOLOGY-MAX-SPOUT-PENDING)] (if p (* p num-tasks)))) (defn init-spout-wait-strategy [storm-conf] (let [ret (-> storm-conf (get TOPOLOGY-SPOUT-WAIT-STRATEGY) new-instance)] (.prepare ret storm-conf) ret )) (defmethod mk-threads :spout [executor-data task-datas] (let [{:keys [storm-conf component-id worker-context transfer-fn report-error sampler open-or-prepare-was-called?]} executor-data ^ISpoutWaitStrategy spout-wait-strategy (init-spout-wait-strategy storm-conf) max-spout-pending (executor-max-spout-pending storm-conf (count task-datas)) ^Integer max-spout-pending (if max-spout-pending (int max-spout-pending)) last-active (atom false) spouts (ArrayList. (map :object (vals task-datas))) rand (Random. (Utils/secureRandomLong)) pending (RotatingMap. 2 ;; microoptimize for performance of .size method (reify RotatingMap$ExpiredCallback (expire [this msg-id [task-id spout-id tuple-info start-time-ms]] (let [time-delta (if start-time-ms (time-delta-ms start-time-ms))] (fail-spout-msg executor-data (get task-datas task-id) spout-id tuple-info time-delta) )))) tuple-action-fn (fn [task-id ^TupleImpl tuple] (let [stream-id (.getSourceStreamId tuple)] (condp = stream-id Constants/SYSTEM_TICK_STREAM_ID (.rotate pending) Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data task-datas tuple) (let [id (.getValue tuple 0) [stored-task-id spout-id tuple-finished-info start-time-ms] (.remove pending id)] (when spout-id (when-not (= stored-task-id task-id) (throw-runtime "Fatal error, mismatched task ids: " task-id " " stored-task-id)) (let [time-delta (if start-time-ms (time-delta-ms start-time-ms))] (condp = stream-id ACKER-ACK-STREAM-ID (ack-spout-msg executor-data (get task-datas task-id) spout-id tuple-finished-info time-delta) ACKER-FAIL-STREAM-ID (fail-spout-msg executor-data (get task-datas task-id) spout-id tuple-finished-info time-delta) ))) ;; TODO: on failure, emit tuple to failure stream )))) receive-queue (:receive-queue executor-data) event-handler (mk-task-receiver executor-data tuple-action-fn) has-ackers? (has-ackers? storm-conf) emitted-count (MutableLong. 0) empty-emit-streak (MutableLong. 0) ;; the overflow buffer is used to ensure that spouts never block when emitting ;; this ensures that the spout can always clear the incoming buffer (acks and fails), which ;; prevents deadlock from occuring across the topology (e.g. Spout -> Bolt -> Acker -> Spout, and all ;; buffers filled up) ;; when the overflow buffer is full, spouts stop calling nextTuple until it's able to clear the overflow buffer ;; this limits the size of the overflow buffer to however many tuples a spout emits in one call of nextTuple, ;; preventing memory issues overflow-buffer (LinkedList.)] [(async-loop (fn [] ;; If topology was started in inactive state, don't call (.open spout) until it's activated first. (while (not @(:storm-active-atom executor-data)) (Thread/sleep 100)) (log-message "Opening spout " component-id ":" (keys task-datas)) (doseq [[task-id task-data] task-datas :let [^ISpout spout-obj (:object task-data) tasks-fn (:tasks-fn task-data) send-spout-msg (fn [out-stream-id values message-id out-task-id] (.increment emitted-count) (let [out-tasks (if out-task-id (tasks-fn out-task-id out-stream-id values) (tasks-fn out-stream-id values)) rooted? (and message-id has-ackers?) root-id (if rooted? (MessageId/generateId rand)) out-ids (fast-list-for [t out-tasks] (if rooted? (MessageId/generateId rand)))] (fast-list-iter [out-task out-tasks id out-ids] (let [tuple-id (if rooted? (MessageId/makeRootId root-id id) (MessageId/makeUnanchored)) out-tuple (TupleImpl. worker-context values task-id out-stream-id tuple-id)] (transfer-fn out-task out-tuple overflow-buffer) )) (if rooted? (do (.put pending root-id [task-id message-id {:stream out-stream-id :values values} (if (sampler) (System/currentTimeMillis))]) (task/send-unanchored task-data ACKER-INIT-STREAM-ID [root-id (bit-xor-vals out-ids) task-id] overflow-buffer)) (when message-id (ack-spout-msg executor-data task-data message-id {:stream out-stream-id :values values} (if (sampler) 0)))) (or out-tasks []) ))]] (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf (:user-context task-data)) (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data) :receive receive-queue} storm-conf (:user-context task-data)) (.open spout-obj storm-conf (:user-context task-data) (SpoutOutputCollector. (reify ISpoutOutputCollector (^List emit [this ^String stream-id ^List tuple ^Object message-id] (send-spout-msg stream-id tuple message-id nil) ) (^void emitDirect [this ^int out-task-id ^String stream-id ^List tuple ^Object message-id] (send-spout-msg stream-id tuple message-id out-task-id) ) (reportError [this error] (report-error error) ))))) (reset! open-or-prepare-was-called? true) (log-message "Opened spout " component-id ":" (keys task-datas)) (setup-metrics! executor-data) (disruptor/consumer-started! (:receive-queue executor-data)) (fn [] ;; This design requires that spouts be non-blocking (disruptor/consume-batch receive-queue event-handler) ;; try to clear the overflow-buffer (try-cause (while (not (.isEmpty overflow-buffer)) (let [[out-task out-tuple] (.peek overflow-buffer)] (transfer-fn out-task out-tuple false nil) (.removeFirst overflow-buffer))) (catch InsufficientCapacityException e )) (let [active? @(:storm-active-atom executor-data) curr-count (.get emitted-count)] (if (and (.isEmpty overflow-buffer) (or (not max-spout-pending) (< (.size pending) max-spout-pending))) (if active? (do (when-not @last-active (reset! last-active true) (log-message "Activating spout " component-id ":" (keys task-datas)) (fast-list-iter [^ISpout spout spouts] (.activate spout))) (fast-list-iter [^ISpout spout spouts] (.nextTuple spout))) (do (when @last-active (reset! last-active false) (log-message "Deactivating spout " component-id ":" (keys task-datas)) (fast-list-iter [^ISpout spout spouts] (.deactivate spout))) ;; TODO: log that it's getting throttled (Time/sleep 100)))) (if (and (= curr-count (.get emitted-count)) active?) (do (.increment empty-emit-streak) (.emptyEmit spout-wait-strategy (.get empty-emit-streak))) (.set empty-emit-streak 0) )) 0)) :kill-fn (:report-error-and-die executor-data) :factory? true :thread-name component-id)])) (defn- tuple-time-delta! [^TupleImpl tuple] (let [ms (.getProcessSampleStartTime tuple)] (if ms (time-delta-ms ms)))) (defn- tuple-execute-time-delta! [^TupleImpl tuple] (let [ms (.getExecuteSampleStartTime tuple)] (if ms (time-delta-ms ms)))) (defn put-xor! [^Map pending key id] (let [curr (or (.get pending key) (long 0))] (.put pending key (bit-xor curr id)))) (defmethod mk-threads :bolt [executor-data task-datas] (let [execute-sampler (mk-stats-sampler (:storm-conf executor-data)) executor-stats (:stats executor-data) {:keys [storm-conf component-id worker-context transfer-fn report-error sampler open-or-prepare-was-called?]} executor-data rand (Random. (Utils/secureRandomLong)) tuple-action-fn (fn [task-id ^TupleImpl tuple] ;; synchronization needs to be done with a key provided by this bolt, otherwise: ;; spout 1 sends synchronization (s1), dies, same spout restarts somewhere else, sends synchronization (s2) and incremental update. s2 and update finish before s1 -> lose the incremental update ;; TODO: for state sync, need to first send sync messages in a loop and receive tuples until synchronization ;; buffer other tuples until fully synchronized, then process all of those tuples ;; then go into normal loop ;; spill to disk? ;; could be receiving incremental updates while waiting for sync or even a partial sync because of another failed task ;; should remember sync requests and include a random sync id in the request. drop anything not related to active sync requests ;; or just timeout the sync messages that are coming in until full sync is hit from that task ;; need to drop incremental updates from tasks where waiting for sync. otherwise, buffer the incremental updates ;; TODO: for state sync, need to check if tuple comes from state spout. if so, update state ;; TODO: how to handle incremental updates as well as synchronizations at same time ;; TODO: need to version tuples somehow ;;(log-debug "Received tuple " tuple " at task " task-id) ;; need to do it this way to avoid reflection (let [stream-id (.getSourceStreamId tuple)] (condp = stream-id Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data task-datas tuple) (let [task-data (get task-datas task-id) ^IBolt bolt-obj (:object task-data) user-context (:user-context task-data) sampler? (sampler) execute-sampler? (execute-sampler) now (if (or sampler? execute-sampler?) (System/currentTimeMillis))] (when sampler? (.setProcessSampleStartTime tuple now)) (when execute-sampler? (.setExecuteSampleStartTime tuple now)) (.execute bolt-obj tuple) (let [delta (tuple-execute-time-delta! tuple)] (task/apply-hooks user-context .boltExecute (BoltExecuteInfo. tuple task-id delta)) (when delta (builtin-metrics/bolt-execute-tuple! (:builtin-metrics task-data) executor-stats (.getSourceComponent tuple) (.getSourceStreamId tuple) delta) (stats/bolt-execute-tuple! executor-stats (.getSourceComponent tuple) (.getSourceStreamId tuple) delta)))))))] ;; TODO: can get any SubscribedState objects out of the context now [(async-loop (fn [] ;; If topology was started in inactive state, don't call prepare bolt until it's activated first. (while (not @(:storm-active-atom executor-data)) (Thread/sleep 100)) (log-message "Preparing bolt " component-id ":" (keys task-datas)) (doseq [[task-id task-data] task-datas :let [^IBolt bolt-obj (:object task-data) tasks-fn (:tasks-fn task-data) user-context (:user-context task-data) bolt-emit (fn [stream anchors values task] (let [out-tasks (if task (tasks-fn task stream values) (tasks-fn stream values))] (fast-list-iter [t out-tasks] (let [anchors-to-ids (HashMap.)] (fast-list-iter [^TupleImpl a anchors] (let [root-ids (-> a .getMessageId .getAnchorsToIds .keySet)] (when (pos? (count root-ids)) (let [edge-id (MessageId/generateId rand)] (.updateAckVal a edge-id) (fast-list-iter [root-id root-ids] (put-xor! anchors-to-ids root-id edge-id)) )))) (transfer-fn t (TupleImpl. worker-context values task-id stream (MessageId/makeId anchors-to-ids))))) (or out-tasks [])))]] (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context) (if (= component-id Constants/SYSTEM_COMPONENT_ID) (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data) :receive (:receive-queue executor-data) :transfer (:transfer-queue (:worker executor-data))} storm-conf user-context) (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data) :receive (:receive-queue executor-data)} storm-conf user-context) ) (.prepare bolt-obj storm-conf user-context (OutputCollector. (reify IOutputCollector (emit [this stream anchors values] (bolt-emit stream anchors values nil)) (emitDirect [this task stream anchors values] (bolt-emit stream anchors values task)) (^void ack [this ^Tuple tuple] (let [^TupleImpl tuple tuple ack-val (.getAckVal tuple)] (fast-map-iter [[root id] (.. tuple getMessageId getAnchorsToIds)] (task/send-unanchored task-data ACKER-ACK-STREAM-ID [root (bit-xor id ack-val)]) )) (let [delta (tuple-time-delta! tuple)] (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta)) (when delta (builtin-metrics/bolt-acked-tuple! (:builtin-metrics task-data) executor-stats (.getSourceComponent tuple) (.getSourceStreamId tuple) delta) (stats/bolt-acked-tuple! executor-stats (.getSourceComponent tuple) (.getSourceStreamId tuple) delta)))) (^void fail [this ^Tuple tuple] (fast-list-iter [root (.. tuple getMessageId getAnchors)] (task/send-unanchored task-data ACKER-FAIL-STREAM-ID [root])) (let [delta (tuple-time-delta! tuple)] (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta)) (when delta (builtin-metrics/bolt-failed-tuple! (:builtin-metrics task-data) executor-stats (.getSourceComponent tuple) (.getSourceStreamId tuple)) (stats/bolt-failed-tuple! executor-stats (.getSourceComponent tuple) (.getSourceStreamId tuple) delta)))) (reportError [this error] (report-error error) ))))) (reset! open-or-prepare-was-called? true) (log-message "Prepared bolt " component-id ":" (keys task-datas)) (setup-metrics! executor-data) (let [receive-queue (:receive-queue executor-data) event-handler (mk-task-receiver executor-data tuple-action-fn)] (disruptor/consumer-started! receive-queue) (fn [] (disruptor/consume-batch-when-available receive-queue event-handler) 0))) :kill-fn (:report-error-and-die executor-data) :factory? true :thread-name component-id)])) (defmethod close-component :spout [executor-data spout] (.close spout)) (defmethod close-component :bolt [executor-data bolt] (.cleanup bolt)) ;; TODO: refactor this to be part of an executor-specific map (defmethod mk-executor-stats :spout [_ rate] (stats/mk-spout-stats rate)) (defmethod mk-executor-stats :bolt [_ rate] (stats/mk-bolt-stats rate)) ================================================ FILE: storm-core/src/clj/backtype/storm/daemon/logviewer.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.logviewer (:use compojure.core) (:use [hiccup core page-helpers]) (:use [backtype.storm config util log]) (:use [ring.adapter.jetty :only [run-jetty]]) (:import [org.slf4j LoggerFactory]) (:import [ch.qos.logback.classic Logger]) (:import [org.apache.commons.logging LogFactory]) (:import [org.apache.commons.logging.impl Log4JLogger]) (:import [ch.qos.logback.core FileAppender]) (:import [org.apache.log4j Level]) (:import [java.io File]) (:require [compojure.route :as route] [compojure.handler :as handler] [clojure.string :as string]) (:gen-class)) (defn tail-file [path tail] (let [flen (.length (clojure.java.io/file path)) skip (- flen tail)] (with-open [input (clojure.java.io/input-stream path) output (java.io.ByteArrayOutputStream.)] (if (> skip 0) (.skip input skip)) (let [buffer (make-array Byte/TYPE 1024)] (loop [] (let [size (.read input buffer)] (when (and (pos? size) (< (.size output) tail)) (do (.write output buffer 0 size) (recur)))))) (.toString output)) )) (defn log-root-dir "Given an appender name, as configured, get the parent directory of the appender's log file. Note that if anything goes wrong, this will throw an Error and exit." [appender-name] (let [appender (.getAppender (LoggerFactory/getLogger Logger/ROOT_LOGGER_NAME) appender-name)] (if (and appender-name appender (instance? FileAppender appender)) (.getParent (File. (.getFile appender))) (throw (RuntimeException. "Log viewer could not find configured appender, or the appender is not a FileAppender. Please check that the appender name configured in storm and logback agree."))))) (defn log-page [file tail grep root-dir] (let [path (.getCanonicalPath (File. root-dir file)) tail (if tail (min 10485760 (Integer/parseInt tail)) 10240) tail-string (tail-file path tail)] (if grep (clojure.string/join "\n
" (filter #(.contains % grep) (.split tail-string "\n"))) (.replaceAll tail-string "\n" "\n
")))) (defn log-level-page [name level] (let [log (LogFactory/getLog name)] (if level (if (instance? Log4JLogger log) (.setLevel (.getLogger log) (Level/toLevel level)))) (str "effective log level for " name " is " (.getLevel (.getLogger log))))) (defn log-template [body] (html4 [:head [:title "Storm log viewer"] (include-css "/css/bootstrap-1.1.0.css") (include-css "/css/style.css") (include-js "/js/jquery-1.6.2.min.js") (include-js "/js/jquery.tablesorter.min.js") (include-js "/js/jquery.cookies.2.2.0.min.js") (include-js "/js/script.js") ] [:body (seq body) ])) (defroutes log-routes (GET "/log" [:as req & m] (log-template (log-page (:file m) (:tail m) (:grep m) (:log-root req)))) (GET "/loglevel" [:as {cookies :cookies} & m] (log-template (log-level-page (:name m) (:level m)))) (route/resources "/") (route/not-found "Page not found")) (def logapp (handler/site log-routes) ) (defn conf-middleware "For passing the storm configuration with each request." [app log-root] (fn [req] (app (assoc req :log-root log-root)))) (defn start-logviewer [port log-root] (run-jetty (conf-middleware logapp log-root) {:port port})) (defn -main [] (let [conf (read-storm-config) log-root (log-root-dir (conf LOGVIEWER-APPENDER-NAME))] (start-logviewer (int (conf LOGVIEWER-PORT)) log-root))) ================================================ FILE: storm-core/src/clj/backtype/storm/daemon/nimbus.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.nimbus (: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 [java.nio ByteBuffer]) (:import [java.io FileNotFoundException]) (:import [java.nio.channels Channels WritableByteChannel]) (:use [backtype.storm.scheduler.DefaultScheduler]) (:import [backtype.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails]) (:use [backtype.storm bootstrap util]) (:use [backtype.storm.daemon common]) (:gen-class :methods [^{:static true} [launch [backtype.storm.scheduler.INimbus] void]])) (bootstrap) (defn file-cache-map [conf] (TimeCacheMap. (int (conf NIMBUS-FILE-COPY-EXPIRATION-SECS)) (reify TimeCacheMap$ExpiredCallback (expire [this id stream] (.close stream) )) )) (defn mk-scheduler [conf inimbus] (let [forced-scheduler (.getForcedScheduler inimbus) scheduler (cond forced-scheduler (do (log-message "Using forced scheduler from INimbus " (class forced-scheduler)) forced-scheduler) (conf STORM-SCHEDULER) (do (log-message "Using custom scheduler: " (conf STORM-SCHEDULER)) (-> (conf STORM-SCHEDULER) new-instance)) :else (do (log-message "Using default scheduler") (DefaultScheduler.)))] (.prepare scheduler conf) scheduler )) (defn nimbus-data [conf inimbus] (let [forced-scheduler (.getForcedScheduler inimbus)] {:conf conf :inimbus inimbus :submitted-count (atom 0) :storm-cluster-state (cluster/mk-storm-cluster-state conf) :submit-lock (Object.) :heartbeats-cache (atom {}) :downloaders (file-cache-map conf) :uploaders (file-cache-map conf) :uptime (uptime-computer) :validator (new-instance (conf NIMBUS-TOPOLOGY-VALIDATOR)) :timer (mk-timer :kill-fn (fn [t] (log-error t "Error when processing event") (halt-process! 20 "Error when processing an event") )) :scheduler (mk-scheduler conf inimbus) })) (defn inbox [nimbus] (master-inbox (:conf nimbus))) (defn- read-storm-conf [conf storm-id] (let [stormroot (master-stormdist-root conf storm-id)] (merge conf (Utils/deserialize (FileUtils/readFileToByteArray (File. (master-stormconf-path stormroot)) ))))) (defn set-topology-status! [nimbus storm-id status] (let [storm-cluster-state (:storm-cluster-state nimbus)] (.update-storm! storm-cluster-state storm-id {:status status}) (log-message "Updated " storm-id " with status " status) )) (declare delay-event) (declare mk-assignments) (defn kill-transition [nimbus storm-id] (fn [kill-time] (let [delay (if kill-time kill-time (get (read-storm-conf (:conf nimbus) storm-id) TOPOLOGY-MESSAGE-TIMEOUT-SECS))] (delay-event nimbus storm-id delay :remove) {:type :killed :kill-time-secs delay}) )) (defn rebalance-transition [nimbus storm-id status] (fn [time num-workers executor-overrides] (let [delay (if time time (get (read-storm-conf (:conf nimbus) storm-id) TOPOLOGY-MESSAGE-TIMEOUT-SECS))] (delay-event nimbus storm-id delay :do-rebalance) {:type :rebalancing :delay-secs delay :old-status status :num-workers num-workers :executor-overrides executor-overrides }))) (defn do-rebalance [nimbus storm-id status] (.update-storm! (:storm-cluster-state nimbus) storm-id (assoc-non-nil {:component->executors (:executor-overrides status)} :num-workers (:num-workers status))) (mk-assignments nimbus :scratch-topology-id storm-id)) (defn state-transitions [nimbus storm-id status] {:active {:inactivate :inactive :activate nil :rebalance (rebalance-transition nimbus storm-id status) :kill (kill-transition nimbus storm-id) } :inactive {:activate :active :inactivate nil :rebalance (rebalance-transition nimbus storm-id status) :kill (kill-transition nimbus storm-id) } :killed {:startup (fn [] (delay-event nimbus storm-id (:kill-time-secs status) :remove) nil) :kill (kill-transition nimbus storm-id) :remove (fn [] (log-message "Killing topology: " storm-id) (.remove-storm! (:storm-cluster-state nimbus) storm-id) nil) } :rebalancing {:startup (fn [] (delay-event nimbus storm-id (:delay-secs status) :do-rebalance) nil) :kill (kill-transition nimbus storm-id) :do-rebalance (fn [] (do-rebalance nimbus storm-id status) (:old-status status)) }}) (defn topology-status [nimbus storm-id] (-> nimbus :storm-cluster-state (.storm-base storm-id nil) :status)) (defn transition! ([nimbus storm-id event] (transition! nimbus storm-id event false)) ([nimbus storm-id event error-on-no-transition?] (locking (:submit-lock nimbus) (let [system-events #{:startup} [event & event-args] (if (keyword? event) [event] event) status (topology-status nimbus storm-id)] ;; handles the case where event was scheduled but topology has been removed (if-not status (log-message "Cannot apply event " event " to " storm-id " because topology no longer exists") (let [get-event (fn [m e] (if (contains? m e) (m e) (let [msg (str "No transition for event: " event ", status: " status, " storm-id: " storm-id)] (if error-on-no-transition? (throw-runtime msg) (do (when-not (contains? system-events event) (log-message msg)) nil)) ))) transition (-> (state-transitions nimbus storm-id status) (get (:type status)) (get-event event)) transition (if (or (nil? transition) (keyword? transition)) (fn [] transition) transition) new-status (apply transition event-args) new-status (if (keyword? new-status) {:type new-status} new-status)] (when new-status (set-topology-status! nimbus storm-id new-status))))) ))) (defn transition-name! [nimbus storm-name event & args] (let [storm-id (get-storm-id (:storm-cluster-state nimbus) storm-name)] (when-not storm-id (throw (NotAliveException. storm-name))) (apply transition! nimbus storm-id event args))) (defn delay-event [nimbus storm-id delay-secs event] (log-message "Delaying event " event " for " delay-secs " secs for " storm-id) (schedule (:timer nimbus) delay-secs #(transition! nimbus storm-id event false) )) ;; active -> reassign in X secs ;; killed -> wait kill time then shutdown ;; active -> reassign in X secs ;; inactive -> nothing ;; rebalance -> wait X seconds then rebalance ;; swap... (need to handle kill during swap, etc.) ;; event transitions are delayed by timer... anything else that comes through (e.g. a kill) override the transition? or just disable other transitions during the transition? (defmulti setup-jar cluster-mode) (defmulti clean-inbox cluster-mode) ;; swapping design ;; -- need 2 ports per worker (swap port and regular port) ;; -- topology that swaps in can use all the existing topologies swap ports, + unused worker slots ;; -- how to define worker resources? port range + number of workers? ;; Monitoring (or by checking when nodes go down or heartbeats aren't received): ;; 1. read assignment ;; 2. see which executors/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 executors and finally remove assignments) (defn- assigned-slots "Returns a map from node-id to a set of ports" [storm-cluster-state] (let [assignments (.assignments storm-cluster-state nil) ] (defaulted (apply merge-with set/union (for [a assignments [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)] {node #{port}} )) {}) )) (defn- all-supervisor-info ([storm-cluster-state] (all-supervisor-info storm-cluster-state nil)) ([storm-cluster-state callback] (let [supervisor-ids (.supervisors storm-cluster-state callback)] (into {} (mapcat (fn [id] (if-let [info (.supervisor-info storm-cluster-state id)] [[id info]] )) supervisor-ids)) ))) (defn- all-scheduling-slots [nimbus topologies missing-assignment-topologies] (let [storm-cluster-state (:storm-cluster-state nimbus) ^INimbus inimbus (:inimbus nimbus) supervisor-infos (all-supervisor-info storm-cluster-state nil) supervisor-details (dofor [[id info] supervisor-infos] (SupervisorDetails. id (:meta info))) ret (.allSlotsAvailableForScheduling inimbus supervisor-details topologies (set missing-assignment-topologies) ) ] (dofor [^WorkerSlot slot ret] [(.getNodeId slot) (.getPort slot)] ))) (defn- optimize-topology [topology] ;; TODO: create new topology by collapsing bolts into CompoundSpout ;; and CompoundBolt ;; need to somehow maintain stream/component ids inside tuples topology) (defn- setup-storm-code [conf storm-id tmp-jar-location storm-conf topology] (let [stormroot (master-stormdist-root conf storm-id)] (FileUtils/forceMkdir (File. stormroot)) (FileUtils/cleanDirectory (File. stormroot)) (setup-jar conf tmp-jar-location stormroot) (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology)) (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/serialize storm-conf)) )) (defn- read-storm-topology [conf storm-id] (let [stormroot (master-stormdist-root conf storm-id)] (Utils/deserialize (FileUtils/readFileToByteArray (File. (master-stormcode-path stormroot)) )))) (declare compute-executor->component) (defn read-topology-details [nimbus storm-id] (let [conf (:conf nimbus) storm-base (.storm-base (:storm-cluster-state nimbus) storm-id nil) topology-conf (read-storm-conf conf storm-id) topology (read-storm-topology conf storm-id) executor->component (->> (compute-executor->component nimbus storm-id) (map-key (fn [[start-task end-task]] (ExecutorDetails. (int start-task) (int end-task)))))] (TopologyDetails. storm-id topology-conf topology (:num-workers storm-base) executor->component ))) ;; Does not assume that clocks are synchronized. Executor heartbeat is only used so that ;; nimbus knows when it's received a new heartbeat. All timing is done by nimbus and ;; tracked through heartbeat-cache (defn- update-executor-cache [curr hb timeout] (let [reported-time (:time-secs hb) {last-nimbus-time :nimbus-time last-reported-time :executor-reported-time} curr reported-time (cond reported-time reported-time last-reported-time last-reported-time :else 0) nimbus-time (if (or (not last-nimbus-time) (not= last-reported-time reported-time)) (current-time-secs) last-nimbus-time )] {:is-timed-out (and nimbus-time (>= (time-delta nimbus-time) timeout)) :nimbus-time nimbus-time :executor-reported-time reported-time})) (defn update-heartbeat-cache [cache executor-beats all-executors timeout] (let [cache (select-keys cache all-executors)] (into {} (for [executor all-executors :let [curr (cache executor)]] [executor (update-executor-cache curr (get executor-beats executor) timeout)] )))) (defn update-heartbeats! [nimbus storm-id all-executors existing-assignment] (log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors)) (let [storm-cluster-state (:storm-cluster-state nimbus) executor-beats (.executor-beats storm-cluster-state storm-id (:executor->node+port existing-assignment)) cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id) executor-beats all-executors ((:conf nimbus) NIMBUS-TASK-TIMEOUT-SECS))] (swap! (:heartbeats-cache nimbus) assoc storm-id cache))) (defn- update-all-heartbeats! [nimbus existing-assignments topology->executors] "update all the heartbeats for all the topologies's executors" (doseq [[tid assignment] existing-assignments :let [all-executors (topology->executors tid)]] (update-heartbeats! nimbus tid all-executors assignment))) (defn- alive-executors [nimbus ^TopologyDetails topology-details all-executors existing-assignment] (log-debug "Computing alive executors for " (.getId topology-details) "\n" "Executors: " (pr-str all-executors) "\n" "Assignment: " (pr-str existing-assignment) "\n" "Heartbeat cache: " (pr-str (@(:heartbeats-cache nimbus) (.getId topology-details))) ) ;; TODO: need to consider all executors associated with a dead executor (in same slot) dead as well, ;; don't just rely on heartbeat being the same (let [conf (:conf nimbus) storm-id (.getId topology-details) executor-start-times (:executor->start-time-secs existing-assignment) heartbeats-cache (@(:heartbeats-cache nimbus) storm-id)] (->> all-executors (filter (fn [executor] (let [start-time (get executor-start-times executor) is-timed-out (-> heartbeats-cache (get executor) :is-timed-out)] (if (and start-time (or (< (time-delta start-time) (conf NIMBUS-TASK-LAUNCH-SECS)) (not is-timed-out) )) true (do (log-message "Executor " storm-id ":" executor " not alive") false)) ))) doall))) (defn- to-executor-id [task-ids] [(first task-ids) (last task-ids)]) (defn- compute-executors [nimbus storm-id] (let [conf (:conf nimbus) storm-base (.storm-base (:storm-cluster-state nimbus) storm-id nil) component->executors (:component->executors storm-base) storm-conf (read-storm-conf conf storm-id) topology (read-storm-topology conf storm-id) task->component (storm-task-info topology storm-conf)] (->> (storm-task-info topology storm-conf) reverse-map (map-val sort) (join-maps component->executors) (map-val (partial apply partition-fixed)) (mapcat second) (map to-executor-id) ))) (defn- compute-executor->component [nimbus storm-id] (let [conf (:conf nimbus) executors (compute-executors nimbus storm-id) topology (read-storm-topology conf storm-id) storm-conf (read-storm-conf conf storm-id) task->component (storm-task-info topology storm-conf) executor->component (into {} (for [executor executors :let [start-task (first executor) component (task->component start-task)]] {executor component}))] executor->component)) (defn- compute-topology->executors [nimbus storm-ids] "compute a topology-id -> executors map" (into {} (for [tid storm-ids] {tid (set (compute-executors nimbus tid))}))) (defn- compute-topology->alive-executors [nimbus existing-assignments topologies topology->executors scratch-topology-id] "compute a topology-id -> alive executors map" (into {} (for [[tid assignment] existing-assignments :let [topology-details (.getById topologies tid) all-executors (topology->executors tid) alive-executors (if (and scratch-topology-id (= scratch-topology-id tid)) all-executors (set (alive-executors nimbus topology-details all-executors assignment)))]] {tid alive-executors}))) (defn- compute-supervisor->dead-ports [nimbus existing-assignments topology->executors topology->alive-executors] (let [dead-slots (into [] (for [[tid assignment] existing-assignments :let [all-executors (topology->executors tid) alive-executors (topology->alive-executors tid) dead-executors (set/difference all-executors alive-executors) dead-slots (->> (:executor->node+port assignment) (filter #(contains? dead-executors (first %))) vals)]] dead-slots)) supervisor->dead-ports (->> dead-slots (apply concat) (map (fn [[sid port]] {sid #{port}})) (apply (partial merge-with set/union)))] (or supervisor->dead-ports {}))) (defn- compute-topology->scheduler-assignment [nimbus existing-assignments topology->alive-executors] "convert assignment information in zk to SchedulerAssignment, so it can be used by scheduler api." (into {} (for [[tid assignment] existing-assignments :let [alive-executors (topology->alive-executors tid) executor->node+port (:executor->node+port assignment) executor->slot (into {} (for [[executor [node port]] executor->node+port] ;; filter out the dead executors (if (contains? alive-executors executor) {(ExecutorDetails. (first executor) (second executor)) (WorkerSlot. node port)} {})))]] {tid (SchedulerAssignmentImpl. tid executor->slot)}))) (defn- read-all-supervisor-details [nimbus all-scheduling-slots supervisor->dead-ports] "return a map: {topology-id SupervisorDetails}" (let [storm-cluster-state (:storm-cluster-state nimbus) supervisor-infos (all-supervisor-info storm-cluster-state) nonexistent-supervisor-slots (apply dissoc all-scheduling-slots (keys supervisor-infos)) all-supervisor-details (into {} (for [[sid supervisor-info] supervisor-infos :let [hostname (:hostname supervisor-info) scheduler-meta (:scheduler-meta supervisor-info) dead-ports (supervisor->dead-ports sid) ;; hide the dead-ports from the all-ports ;; these dead-ports can be reused in next round of assignments all-ports (-> (get all-scheduling-slots sid) (set/difference dead-ports) ((fn [ports] (map int ports)))) supervisor-details (SupervisorDetails. sid hostname scheduler-meta all-ports)]] {sid supervisor-details}))] (merge all-supervisor-details (into {} (for [[sid ports] nonexistent-supervisor-slots] [sid (SupervisorDetails. sid nil ports)])) ))) (defn- compute-topology->executor->node+port [scheduler-assignments] "convert {topology-id -> SchedulerAssignment} to {topology-id -> {executor [node port]}}" (map-val (fn [^SchedulerAssignment assignment] (->> assignment .getExecutorToSlot (#(into {} (for [[^ExecutorDetails executor ^WorkerSlot slot] %] {[(.getStartTask executor) (.getEndTask executor)] [(.getNodeId slot) (.getPort slot)]}))))) scheduler-assignments)) ;; NEW NOTES ;; only assign to supervisors who are there and haven't timed out ;; need to reassign workers with executors that have timed out (will this make it brittle?) ;; need to read in the topology and storm-conf from disk ;; if no slots available and no slots used by this storm, just skip and do nothing ;; otherwise, package rest of executors into available slots (up to how much it needs) ;; in the future could allocate executors intelligently (so that "close" tasks reside on same machine) ;; TODO: slots that have dead executor should be reused as long as supervisor is active ;; (defn- assigned-slots-from-scheduler-assignments [topology->assignment] ;; (->> topology->assignment ;; vals ;; (map (fn [^SchedulerAssignment a] (.getExecutorToSlot a))) ;; (mapcat vals) ;; (map (fn [^WorkerSlot s] {(.getNodeId s) #{(.getPort s)}})) ;; (apply merge-with set/union) ;; )) (defn num-used-workers [^SchedulerAssignment scheduler-assignment] (if scheduler-assignment (count (.getSlots scheduler-assignment)) 0 )) ;; public so it can be mocked out (defn compute-new-topology->executor->node+port [nimbus existing-assignments topologies scratch-topology-id] (let [conf (:conf nimbus) storm-cluster-state (:storm-cluster-state nimbus) topology->executors (compute-topology->executors nimbus (keys existing-assignments)) ;; update the executors heartbeats first. _ (update-all-heartbeats! nimbus existing-assignments topology->executors) topology->alive-executors (compute-topology->alive-executors nimbus existing-assignments topologies topology->executors scratch-topology-id) supervisor->dead-ports (compute-supervisor->dead-ports nimbus existing-assignments topology->executors topology->alive-executors) topology->scheduler-assignment (compute-topology->scheduler-assignment nimbus existing-assignments topology->alive-executors) missing-assignment-topologies (->> topologies .getTopologies (map (memfn getId)) (filter (fn [t] (let [alle (get topology->executors t) alivee (get topology->alive-executors t)] (or (empty? alle) (not= alle alivee) (< (-> topology->scheduler-assignment (get t) num-used-workers ) (-> topologies (.getById t) .getNumWorkers) )) )))) all-scheduling-slots (->> (all-scheduling-slots nimbus topologies missing-assignment-topologies) (map (fn [[node-id port]] {node-id #{port}})) (apply merge-with set/union)) supervisors (read-all-supervisor-details nimbus all-scheduling-slots supervisor->dead-ports) cluster (Cluster. (:inimbus nimbus) supervisors topology->scheduler-assignment) ;; call scheduler.schedule to schedule all the topologies ;; the new assignments for all the topologies are in the cluster object. _ (.schedule (:scheduler nimbus) topologies cluster) new-scheduler-assignments (.getAssignments cluster) ;; add more information to convert SchedulerAssignment to Assignment new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)] ;; print some useful information. (doseq [[topology-id executor->node+port] new-topology->executor->node+port :let [old-executor->node+port (-> topology-id existing-assignments :executor->node+port) reassignment (filter (fn [[executor node+port]] (and (contains? old-executor->node+port executor) (not (= node+port (old-executor->node+port executor))))) executor->node+port)]] (when-not (empty? reassignment) (let [new-slots-cnt (count (set (vals executor->node+port))) reassign-executors (keys reassignment)] (log-message "Reassigning " topology-id " to " new-slots-cnt " slots") (log-message "Reassign executors: " (vec reassign-executors))))) new-topology->executor->node+port)) (defn changed-executors [executor->node+port new-executor->node+port] (let [slot-assigned (reverse-map executor->node+port) new-slot-assigned (reverse-map new-executor->node+port) brand-new-slots (map-diff slot-assigned new-slot-assigned)] (apply concat (vals brand-new-slots)) )) (defn newly-added-slots [existing-assignment new-assignment] (let [old-slots (-> (:executor->node+port existing-assignment) vals set) new-slots (-> (:executor->node+port new-assignment) vals set)] (set/difference new-slots old-slots))) (defn basic-supervisor-details-map [storm-cluster-state] (let [infos (all-supervisor-info storm-cluster-state)] (->> infos (map (fn [[id info]] [id (SupervisorDetails. id (:hostname info) (:scheduler-meta info) nil)])) (into {})))) (defn- to-worker-slot [[node port]] (WorkerSlot. node port)) ;; get existing assignment (just the executor->node+port map) -> default to {} ;; filter out ones which have a executor timeout ;; figure out available slots on cluster. add to that the used valid slots to get total slots. figure out how many executors should be in each slot (e.g., 4, 4, 4, 5) ;; only keep existing slots that satisfy one of those slots. for rest, reassign them across remaining slots ;; edge case for slots with no executor timeout but with supervisor timeout... just treat these as valid slots that can be reassigned to. worst comes to worse the executor will timeout and won't assign here next time around (defnk mk-assignments [nimbus :scratch-topology-id nil] (let [conf (:conf nimbus) storm-cluster-state (:storm-cluster-state nimbus) ^INimbus inimbus (:inimbus nimbus) ;; read all the topologies topology-ids (.active-storms storm-cluster-state) topologies (into {} (for [tid topology-ids] {tid (read-topology-details nimbus tid)})) topologies (Topologies. topologies) ;; read all the assignments assigned-topology-ids (.assignments storm-cluster-state nil) existing-assignments (into {} (for [tid assigned-topology-ids] ;; for the topology which wants rebalance (specified by the scratch-topology-id) ;; we exclude its assignment, meaning that all the slots occupied by its assignment ;; will be treated as free slot in the scheduler code. (when (or (nil? scratch-topology-id) (not= tid scratch-topology-id)) {tid (.assignment-info storm-cluster-state tid nil)}))) ;; make the new assignments for topologies topology->executor->node+port (compute-new-topology->executor->node+port nimbus existing-assignments topologies scratch-topology-id) now-secs (current-time-secs) basic-supervisor-details-map (basic-supervisor-details-map storm-cluster-state) ;; construct the final Assignments by adding start-times etc into it new-assignments (into {} (for [[topology-id executor->node+port] topology->executor->node+port :let [existing-assignment (get existing-assignments topology-id) all-nodes (->> executor->node+port vals (map first) set) node->host (->> all-nodes (mapcat (fn [node] (if-let [host (.getHostName inimbus basic-supervisor-details-map node)] [[node host]] ))) (into {})) all-node->host (merge (:node->host existing-assignment) node->host) reassign-executors (changed-executors (:executor->node+port existing-assignment) executor->node+port) start-times (merge (:executor->start-time-secs existing-assignment) (into {} (for [id reassign-executors] [id now-secs] )))]] {topology-id (Assignment. (master-stormdist-root conf topology-id) (select-keys all-node->host all-nodes) executor->node+port start-times)}))] ;; tasks figure out what tasks to talk to by looking at topology at runtime ;; only log/set when there's been a change to the assignment (doseq [[topology-id assignment] new-assignments :let [existing-assignment (get existing-assignments topology-id) topology-details (.getById topologies topology-id)]] (if (= existing-assignment assignment) (log-debug "Assignment for " topology-id " hasn't changed") (do (log-message "Setting new assignment for topology id " topology-id ": " (pr-str assignment)) (.set-assignment! storm-cluster-state topology-id assignment) ))) (->> new-assignments (map (fn [[topology-id assignment]] (let [existing-assignment (get existing-assignments topology-id)] [topology-id (map to-worker-slot (newly-added-slots existing-assignment assignment))] ))) (into {}) (.assignSlots inimbus topologies)) )) (defn- start-storm [nimbus storm-name storm-id topology-initial-status] {:pre [(#{:active :inactive} topology-initial-status)]} (let [storm-cluster-state (:storm-cluster-state nimbus) conf (:conf nimbus) storm-conf (read-storm-conf conf storm-id) topology (system-topology! storm-conf (read-storm-topology conf storm-id)) num-executors (->> (all-components topology) (map-val num-start-executors))] (log-message "Activating " storm-name ": " storm-id) (.activate-storm! storm-cluster-state storm-id (StormBase. storm-name (current-time-secs) {:type topology-initial-status} (storm-conf TOPOLOGY-WORKERS) num-executors)))) ;; Master: ;; job submit: ;; 1. read which nodes are available ;; 2. set assignments ;; 3. start storm - necessary in case master goes down, when goes back up can remember to take down the storm (2 states: on or off) (defn storm-active? [storm-cluster-state storm-name] (not-nil? (get-storm-id storm-cluster-state storm-name))) (defn check-storm-active! [nimbus storm-name active?] (if (= (not active?) (storm-active? (:storm-cluster-state nimbus) storm-name)) (if active? (throw (NotAliveException. (str storm-name " is not alive"))) (throw (AlreadyAliveException. (str storm-name " is already active")))) )) (defn code-ids [conf] (-> conf master-stormdist-root read-dir-contents set )) (defn cleanup-storm-ids [conf storm-cluster-state] (let [heartbeat-ids (set (.heartbeat-storms storm-cluster-state)) error-ids (set (.error-topologies storm-cluster-state)) code-ids (code-ids conf) assigned-ids (set (.active-storms storm-cluster-state))] (set/difference (set/union heartbeat-ids error-ids code-ids) assigned-ids) )) (defn extract-status-str [base] (let [t (-> base :status :type)] (.toUpperCase (name t)) )) (defn mapify-serializations [sers] (->> sers (map (fn [e] (if (map? e) e {e nil}))) (apply merge) )) (defn- component-parallelism [storm-conf component] (let [storm-conf (merge storm-conf (component-conf component)) num-tasks (or (storm-conf TOPOLOGY-TASKS) (num-start-executors component)) max-parallelism (storm-conf TOPOLOGY-MAX-TASK-PARALLELISM) ] (if max-parallelism (min max-parallelism num-tasks) num-tasks))) (defn normalize-topology [storm-conf ^StormTopology topology] (let [ret (.deepCopy topology)] (doseq [[_ component] (all-components ret)] (.set_json_conf (.get_common component) (->> {TOPOLOGY-TASKS (component-parallelism storm-conf component)} (merge (component-conf component)) to-json ))) ret )) (defn normalize-conf [conf storm-conf ^StormTopology topology] ;; ensure that serializations are same for all tasks no matter what's on ;; the supervisors. this also allows you to declare the serializations as a sequence (let [component-confs (map #(-> (ThriftTopologyUtils/getComponentCommon topology %) .get_json_conf from-json) (ThriftTopologyUtils/getComponentIds topology)) total-conf (merge conf storm-conf) get-merged-conf-val (fn [k merge-fn] (merge-fn (concat (mapcat #(get % k) component-confs) (or (get storm-conf k) (get conf k)))))] ;; topology level serialization registrations take priority ;; that way, if there's a conflict, a user can force which serialization to use ;; append component conf to storm-conf (merge storm-conf {TOPOLOGY-KRYO-DECORATORS (get-merged-conf-val TOPOLOGY-KRYO-DECORATORS distinct) TOPOLOGY-KRYO-REGISTER (get-merged-conf-val TOPOLOGY-KRYO-REGISTER mapify-serializations) TOPOLOGY-ACKER-EXECUTORS (total-conf TOPOLOGY-ACKER-EXECUTORS) TOPOLOGY-MAX-TASK-PARALLELISM (total-conf TOPOLOGY-MAX-TASK-PARALLELISM)}))) (defn do-cleanup [nimbus] (let [storm-cluster-state (:storm-cluster-state nimbus) conf (:conf nimbus) submit-lock (:submit-lock nimbus)] (let [to-cleanup-ids (locking submit-lock (cleanup-storm-ids conf storm-cluster-state))] (when-not (empty? to-cleanup-ids) (doseq [id to-cleanup-ids] (log-message "Cleaning up " id) (.teardown-heartbeats! storm-cluster-state id) (.teardown-topology-errors! storm-cluster-state id) (rmr (master-stormdist-root conf id)) (swap! (:heartbeats-cache nimbus) dissoc id)) )))) (defn- file-older-than? [now seconds file] (<= (+ (.lastModified file) (to-millis seconds)) (to-millis now))) (defn clean-inbox [dir-location seconds] "Deletes jar files in dir older than seconds." (let [now (current-time-secs) pred #(and (.isFile %) (file-older-than? now seconds %)) files (filter pred (file-seq (File. dir-location)))] (doseq [f files] (if (.delete f) (log-message "Cleaning inbox ... deleted: " (.getName f)) ;; This should never happen (log-error "Cleaning inbox ... error deleting: " (.getName f)) )))) (defn cleanup-corrupt-topologies! [nimbus] (let [storm-cluster-state (:storm-cluster-state nimbus) code-ids (set (code-ids (:conf nimbus))) active-topologies (set (.active-storms storm-cluster-state)) corrupt-topologies (set/difference active-topologies code-ids)] (doseq [corrupt corrupt-topologies] (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...") (.remove-storm! storm-cluster-state corrupt) ))) (defn- get-errors [storm-cluster-state storm-id component-id] (->> (.errors storm-cluster-state storm-id component-id) (map #(ErrorInfo. (:error %) (:time-secs %))))) (defn- thriftify-executor-id [[first-task-id last-task-id]] (ExecutorInfo. (int first-task-id) (int last-task-id))) (def DISALLOWED-TOPOLOGY-NAME-STRS #{"/" "." ":" "\\"}) (defn validate-topology-name! [name] (if (some #(.contains name %) DISALLOWED-TOPOLOGY-NAME-STRS) (throw (InvalidTopologyException. (str "Topology name cannot contain any of the following: " (pr-str DISALLOWED-TOPOLOGY-NAME-STRS)))) (if (clojure.string/blank? name) (throw (InvalidTopologyException. ("Topology name cannot be blank")))))) (defn- try-read-storm-conf [conf storm-id] (try-cause (read-storm-conf conf storm-id) (catch FileNotFoundException e (throw (NotAliveException. storm-id))) ) ) (defn- try-read-storm-topology [conf storm-id] (try-cause (read-storm-topology conf storm-id) (catch FileNotFoundException e (throw (NotAliveException. storm-id))) ) ) (defserverfn service-handler [conf inimbus] (.prepare inimbus conf (master-inimbus-dir conf)) (log-message "Starting Nimbus with conf " conf) (let [nimbus (nimbus-data conf inimbus)] (.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf) (cleanup-corrupt-topologies! nimbus) (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))] (transition! nimbus storm-id :startup)) (schedule-recurring (:timer nimbus) 0 (conf NIMBUS-MONITOR-FREQ-SECS) (fn [] (when (conf NIMBUS-REASSIGN) (locking (:submit-lock nimbus) (mk-assignments nimbus))) (do-cleanup nimbus) )) ;; Schedule Nimbus inbox cleaner (schedule-recurring (:timer nimbus) 0 (conf NIMBUS-CLEANUP-INBOX-FREQ-SECS) (fn [] (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS)) )) (reify Nimbus$Iface (^void submitTopologyWithOpts [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology ^SubmitOptions submitOptions] (try (assert (not-nil? submitOptions)) (validate-topology-name! storm-name) (check-storm-active! nimbus storm-name false) (.validate ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) storm-name (from-json serializedConf) topology) (swap! (:submitted-count nimbus) inc) (let [storm-id (str storm-name "-" @(:submitted-count nimbus) "-" (current-time-secs)) storm-conf (normalize-conf conf (-> serializedConf from-json (assoc STORM-ID storm-id) (assoc TOPOLOGY-NAME storm-name)) topology) total-storm-conf (merge conf storm-conf) topology (normalize-topology total-storm-conf topology) topology (if (total-storm-conf TOPOLOGY-OPTIMIZE) (optimize-topology topology) topology) storm-cluster-state (:storm-cluster-state nimbus)] (system-topology! total-storm-conf topology) ;; this validates the structure of the topology (log-message "Received topology submission for " storm-name " with conf " storm-conf) ;; lock protects against multiple topologies being submitted at once and ;; cleanup thread killing topology in b/w assignment and starting the topology (locking (:submit-lock nimbus) (setup-storm-code conf storm-id uploadedJarLocation storm-conf topology) (.setup-heartbeats! storm-cluster-state storm-id) (let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive TopologyInitialStatus/ACTIVE :active}] (start-storm nimbus storm-name storm-id (thrift-status->kw-status (.get_initial_status submitOptions)))) (mk-assignments nimbus))) (catch Throwable e (log-warn-error e "Topology submission exception. (topology name='" storm-name "')") (throw e)))) (^void submitTopology [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology] (.submitTopologyWithOpts this storm-name uploadedJarLocation serializedConf topology (SubmitOptions. TopologyInitialStatus/ACTIVE))) (^void killTopology [this ^String name] (.killTopologyWithOpts this name (KillOptions.))) (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options] (check-storm-active! nimbus storm-name true) (let [wait-amt (if (.is_set_wait_secs options) (.get_wait_secs options) )] (transition-name! nimbus storm-name [:kill wait-amt] true) )) (^void rebalance [this ^String storm-name ^RebalanceOptions options] (check-storm-active! nimbus storm-name true) (let [wait-amt (if (.is_set_wait_secs options) (.get_wait_secs options)) num-workers (if (.is_set_num_workers options) (.get_num_workers options)) executor-overrides (if (.is_set_num_executors options) (.get_num_executors options) {})] (doseq [[c num-executors] executor-overrides] (when (<= num-executors 0) (throw (InvalidTopologyException. "Number of executors must be greater than 0")) )) (transition-name! nimbus storm-name [:rebalance wait-amt num-workers executor-overrides] true) )) (activate [this storm-name] (transition-name! nimbus storm-name :activate true) ) (deactivate [this storm-name] (transition-name! nimbus storm-name :inactivate true)) (beginFileUpload [this] (let [fileloc (str (inbox nimbus) "/stormjar-" (uuid) ".jar")] (.put (:uploaders nimbus) fileloc (Channels/newChannel (FileOutputStream. fileloc))) (log-message "Uploading file from client to " fileloc) fileloc )) (^void uploadChunk [this ^String location ^ByteBuffer chunk] (let [uploaders (:uploaders nimbus) ^WritableByteChannel channel (.get uploaders location)] (when-not channel (throw (RuntimeException. "File for that location does not exist (or timed out)"))) (.write channel chunk) (.put uploaders location channel) )) (^void finishFileUpload [this ^String location] (let [uploaders (:uploaders nimbus) ^WritableByteChannel channel (.get uploaders location)] (when-not channel (throw (RuntimeException. "File for that location does not exist (or timed out)"))) (.close channel) (log-message "Finished uploading file from client: " location) (.remove uploaders location) )) (^String beginFileDownload [this ^String file] (let [is (BufferFileInputStream. file) id (uuid)] (.put (:downloaders nimbus) id is) id )) (^ByteBuffer downloadChunk [this ^String id] (let [downloaders (:downloaders nimbus) ^BufferFileInputStream is (.get downloaders id)] (when-not is (throw (RuntimeException. "Could not find input stream for that id"))) (let [ret (.read is)] (.put downloaders id is) (when (empty? ret) (.remove downloaders id)) (ByteBuffer/wrap ret) ))) (^String getNimbusConf [this] (to-json (:conf nimbus))) (^String getTopologyConf [this ^String id] (to-json (try-read-storm-conf conf id))) (^StormTopology getTopology [this ^String id] (system-topology! (try-read-storm-conf conf id) (try-read-storm-topology conf id))) (^StormTopology getUserTopology [this ^String id] (try-read-storm-topology conf id)) (^ClusterSummary getClusterInfo [this] (let [storm-cluster-state (:storm-cluster-state nimbus) supervisor-infos (all-supervisor-info storm-cluster-state) ;; TODO: need to get the port info about supervisors... ;; in standalone just look at metadata, otherwise just say N/A? supervisor-summaries (dofor [[id info] supervisor-infos] (let [ports (set (:meta info)) ;;TODO: this is only true for standalone ] (SupervisorSummary. (:hostname info) (:uptime-secs info) (count ports) (count (:used-ports info)) id ) )) nimbus-uptime ((:uptime nimbus)) bases (topology-bases storm-cluster-state) topology-summaries (dofor [[id base] bases] (let [assignment (.assignment-info storm-cluster-state id nil)] (TopologySummary. id (:storm-name base) (->> (:executor->node+port assignment) keys (mapcat executor-id->tasks) count) (->> (:executor->node+port assignment) keys count) (->> (:executor->node+port assignment) vals set count) (time-delta (:launch-time-secs base)) (extract-status-str base)) ))] (ClusterSummary. supervisor-summaries nimbus-uptime topology-summaries) )) (^TopologyInfo getTopologyInfo [this ^String storm-id] (let [storm-cluster-state (:storm-cluster-state nimbus) task->component (storm-task-info (try-read-storm-topology conf storm-id) (try-read-storm-conf conf storm-id)) base (.storm-base storm-cluster-state storm-id nil) assignment (.assignment-info storm-cluster-state storm-id nil) beats (.executor-beats storm-cluster-state storm-id (:executor->node+port assignment)) all-components (-> task->component reverse-map keys) errors (->> all-components (map (fn [c] [c (get-errors storm-cluster-state storm-id c)])) (into {})) executor-summaries (dofor [[executor [node port]] (:executor->node+port assignment)] (let [host (-> assignment :node->host (get node)) heartbeat (get beats executor) stats (:stats heartbeat) stats (if stats (stats/thriftify-executor-stats stats))] (doto (ExecutorSummary. (thriftify-executor-id executor) (-> executor first task->component) host port (nil-to-zero (:uptime heartbeat))) (.set_stats stats)) )) ] (TopologyInfo. storm-id (:storm-name base) (time-delta (:launch-time-secs base)) executor-summaries (extract-status-str base) errors ) )) Shutdownable (shutdown [this] (log-message "Shutting down master") (cancel-timer (:timer nimbus)) (.disconnect (:storm-cluster-state nimbus)) (.cleanup (:downloaders nimbus)) (.cleanup (:uploaders nimbus)) (log-message "Shut down master") ) DaemonCommon (waiting? [this] (timer-waiting? (:timer nimbus)))))) (defn launch-server! [conf nimbus] (validate-distributed-mode! conf) (let [service-handler (service-handler conf nimbus) options (-> (TNonblockingServerSocket. (int (conf NIMBUS-THRIFT-PORT))) (THsHaServer$Args.) (.workerThreads 64) (.protocolFactory (TBinaryProtocol$Factory.)) (.processor (Nimbus$Processor. service-handler)) ) server (THsHaServer. options)] (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.shutdown service-handler) (.stop server)))) (log-message "Starting Nimbus server...") (.serve server))) ;; distributed implementation (defmethod setup-jar :distributed [conf tmp-jar-location stormroot] (let [src-file (File. tmp-jar-location)] (if-not (.exists src-file) (throw (IllegalArgumentException. (str tmp-jar-location " to copy to " stormroot " does not exist!")))) (FileUtils/copyFile src-file (File. (master-stormjar-path stormroot))) )) ;; local implementation (defmethod setup-jar :local [conf & args] nil ) (defn -launch [nimbus] (launch-server! (read-storm-config) nimbus)) (defn standalone-nimbus [] (reify INimbus (prepare [this conf local-dir] ) (allSlotsAvailableForScheduling [this supervisors topologies topologies-missing-assignments] (->> supervisors (mapcat (fn [^SupervisorDetails s] (for [p (.getMeta s)] (WorkerSlot. (.getId s) p)))) set )) (assignSlots [this topology slots] ) (getForcedScheduler [this] nil ) (getHostName [this supervisors node-id] (if-let [^SupervisorDetails supervisor (get supervisors node-id)] (.getHost supervisor))) )) (defn -main [] (-launch (standalone-nimbus))) ================================================ FILE: storm-core/src/clj/backtype/storm/daemon/supervisor.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.supervisor (:import [backtype.storm.scheduler ISupervisor]) (:use [backtype.storm bootstrap]) (:use [backtype.storm.daemon common]) (:require [backtype.storm.daemon [worker :as worker]]) (:gen-class :methods [^{:static true} [launch [backtype.storm.scheduler.ISupervisor] void]])) (bootstrap) (defmulti download-storm-code cluster-mode) (defmulti launch-worker (fn [supervisor & _] (cluster-mode (:conf supervisor)))) ;; used as part of a map from port to this (defrecord LocalAssignment [storm-id executors]) (defprotocol SupervisorDaemon (get-id [this]) (get-conf [this]) (shutdown-all-workers [this]) ) (defn- assignments-snapshot [storm-cluster-state callback] (let [storm-ids (.assignments storm-cluster-state callback)] (->> (dofor [sid storm-ids] {sid (.assignment-info storm-cluster-state sid callback)}) (apply merge) (filter-val not-nil?) ))) (defn- read-my-executors [assignments-snapshot storm-id assignment-id] (let [assignment (get assignments-snapshot storm-id) my-executors (filter (fn [[_ [node _]]] (= node assignment-id)) (:executor->node+port assignment)) port-executors (apply merge-with concat (for [[executor [_ port]] my-executors] {port [executor]} ))] (into {} (for [[port executors] port-executors] ;; need to cast to int b/c it might be a long (due to how yaml parses things) ;; doall is to avoid serialization/deserialization problems with lazy seqs [(Integer. port) (LocalAssignment. storm-id (doall executors))] )))) (defn- read-assignments "Returns map from port to struct containing :storm-id and :executors" [assignments-snapshot assignment-id] (->> (dofor [sid (keys assignments-snapshot)] (read-my-executors assignments-snapshot sid assignment-id)) (apply merge-with (fn [& ignored] (throw-runtime "Should not have multiple topologies assigned to one port"))))) (defn- read-storm-code-locations [assignments-snapshot] (map-val :master-code-dir assignments-snapshot)) (defn- read-downloaded-storm-ids [conf] (map #(java.net.URLDecoder/decode %) (read-dir-contents (supervisor-stormdist-root conf))) ) (defn read-worker-heartbeat [conf id] (let [local-state (worker-state conf id)] (.get local-state LS-WORKER-HEARTBEAT) )) (defn my-worker-ids [conf] (read-dir-contents (worker-root conf))) (defn read-worker-heartbeats "Returns map from worker id to heartbeat" [conf] (let [ids (my-worker-ids conf)] (into {} (dofor [id ids] [id (read-worker-heartbeat conf id)])) )) (defn matches-an-assignment? [worker-heartbeat assigned-executors] (let [local-assignment (assigned-executors (:port worker-heartbeat))] (and local-assignment (= (:storm-id worker-heartbeat) (:storm-id local-assignment)) (= (disj (set (:executors worker-heartbeat)) Constants/SYSTEM_EXECUTOR_ID) (set (:executors local-assignment)))))) (defn read-allocated-workers "Returns map from worker id to worker heartbeat. if the heartbeat is nil, then the worker is dead (timed out or never wrote heartbeat)" [supervisor assigned-executors now] (let [conf (:conf supervisor) ^LocalState local-state (:local-state supervisor) id->heartbeat (read-worker-heartbeats conf) approved-ids (set (keys (.get local-state LS-APPROVED-WORKERS)))] (into {} (dofor [[id hb] id->heartbeat] (let [state (cond (or (not (contains? approved-ids id)) (not (matches-an-assignment? hb assigned-executors))) :disallowed (not hb) :not-started (> (- now (:time-secs hb)) (conf SUPERVISOR-WORKER-TIMEOUT-SECS)) :timed-out true :valid)] (log-debug "Worker " id " is " state ": " (pr-str hb) " at supervisor time-secs " now) [id [state hb]] )) ))) (defn- wait-for-worker-launch [conf id start-time] (let [state (worker-state conf id)] (loop [] (let [hb (.get state LS-WORKER-HEARTBEAT)] (when (and (not hb) (< (- (current-time-secs) start-time) (conf SUPERVISOR-WORKER-START-TIMEOUT-SECS) )) (log-message id " still hasn't started") (Time/sleep 500) (recur) ))) (when-not (.get state LS-WORKER-HEARTBEAT) (log-message "Worker " id " failed to start") ))) (defn- wait-for-workers-launch [conf ids] (let [start-time (current-time-secs)] (doseq [id ids] (wait-for-worker-launch conf id start-time)) )) (defn generate-supervisor-id [] (uuid)) (defn try-cleanup-worker [conf id] (try (rmr (worker-heartbeats-root conf id)) ;; this avoids a race condition with worker or subprocess writing pid around same time (rmpath (worker-pids-root conf id)) (rmpath (worker-root conf id)) (catch RuntimeException e (log-warn-error e "Failed to cleanup worker " id ". Will retry later") ))) (defn shutdown-worker [supervisor id] (log-message "Shutting down " (:supervisor-id supervisor) ":" id) (let [conf (:conf supervisor) pids (read-dir-contents (worker-pids-root conf id)) thread-pid (@(:worker-thread-pids-atom supervisor) id)] (when thread-pid (psim/kill-process thread-pid)) (doseq [pid pids] (ensure-process-killed! pid) (rmpath (worker-pid-path conf id pid)) ) (try-cleanup-worker conf id)) (log-message "Shut down " (:supervisor-id supervisor) ":" id)) (defn supervisor-data [conf shared-context ^ISupervisor isupervisor] {:conf conf :shared-context shared-context :isupervisor isupervisor :active (atom true) :uptime (uptime-computer) :worker-thread-pids-atom (atom {}) :storm-cluster-state (cluster/mk-storm-cluster-state conf) :local-state (supervisor-state conf) :supervisor-id (.getSupervisorId isupervisor) :assignment-id (.getAssignmentId isupervisor) :my-hostname (if (contains? conf STORM-LOCAL-HOSTNAME) (conf STORM-LOCAL-HOSTNAME) (local-hostname)) :curr-assignment (atom nil) ;; used for reporting used ports when heartbeating :timer (mk-timer :kill-fn (fn [t] (log-error t "Error when processing event") (halt-process! 20 "Error when processing an event") )) }) (defn sync-processes [supervisor] (let [conf (:conf supervisor) ^LocalState local-state (:local-state supervisor) assigned-executors (defaulted (.get local-state LS-LOCAL-ASSIGNMENTS) {}) now (current-time-secs) allocated (read-allocated-workers supervisor assigned-executors now) keepers (filter-val (fn [[state _]] (= state :valid)) allocated) keep-ports (set (for [[id [_ hb]] keepers] (:port hb))) reassign-executors (select-keys-pred (complement keep-ports) assigned-executors) new-worker-ids (into {} (for [port (keys reassign-executors)] [port (uuid)])) ] ;; 1. to kill are those in allocated that are dead or disallowed ;; 2. kill the ones that should be dead ;; - read pids, kill -9 and individually remove file ;; - rmr heartbeat dir, rmdir pid dir, rmdir id dir (catch exception and log) ;; 3. of the rest, figure out what assignments aren't yet satisfied ;; 4. generate new worker ids, write new "approved workers" to LS ;; 5. create local dir for worker id ;; 5. launch new workers (give worker-id, port, and supervisor-id) ;; 6. wait for workers launch (log-debug "Syncing processes") (log-debug "Assigned executors: " assigned-executors) (log-debug "Allocated: " allocated) (doseq [[id [state heartbeat]] allocated] (when (not= :valid state) (log-message "Shutting down and clearing state for id " id ". Current supervisor time: " now ". State: " state ", Heartbeat: " (pr-str heartbeat)) (shutdown-worker supervisor id) )) (doseq [id (vals new-worker-ids)] (local-mkdirs (worker-pids-root conf id))) (.put local-state LS-APPROVED-WORKERS (merge (select-keys (.get local-state LS-APPROVED-WORKERS) (keys keepers)) (zipmap (vals new-worker-ids) (keys new-worker-ids)) )) (wait-for-workers-launch conf (dofor [[port assignment] reassign-executors] (let [id (new-worker-ids port)] (log-message "Launching worker with assignment " (pr-str assignment) " for this supervisor " (:supervisor-id supervisor) " on port " port " with id " id ) (launch-worker supervisor (:storm-id assignment) port id) id))) )) (defn assigned-storm-ids-from-port-assignments [assignment] (->> assignment vals (map :storm-id) set)) (defn mk-synchronize-supervisor [supervisor sync-processes event-manager processes-event-manager] (fn this [] (let [conf (:conf supervisor) storm-cluster-state (:storm-cluster-state supervisor) ^ISupervisor isupervisor (:isupervisor supervisor) ^LocalState local-state (:local-state supervisor) sync-callback (fn [& ignored] (.add event-manager this)) assignments-snapshot (assignments-snapshot storm-cluster-state sync-callback) storm-code-map (read-storm-code-locations assignments-snapshot) downloaded-storm-ids (set (read-downloaded-storm-ids conf)) all-assignment (read-assignments assignments-snapshot (:assignment-id supervisor)) new-assignment (->> all-assignment (filter-key #(.confirmAssigned isupervisor %))) assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment) existing-assignment (.get local-state LS-LOCAL-ASSIGNMENTS)] (log-debug "Synchronizing supervisor") (log-debug "Storm code map: " storm-code-map) (log-debug "Downloaded storm ids: " downloaded-storm-ids) (log-debug "All assignment: " all-assignment) (log-debug "New assignment: " new-assignment) ;; download code first ;; This might take awhile ;; - should this be done separately from usual monitoring? ;; should we only download when topology is assigned to this supervisor? (doseq [[storm-id master-code-dir] storm-code-map] (when (and (not (downloaded-storm-ids storm-id)) (assigned-storm-ids storm-id)) (log-message "Downloading code for storm id " storm-id " from " master-code-dir) (download-storm-code conf storm-id master-code-dir) (log-message "Finished downloading code for storm id " storm-id " from " master-code-dir) )) (log-debug "Writing new assignment " (pr-str new-assignment)) (doseq [p (set/difference (set (keys existing-assignment)) (set (keys new-assignment)))] (.killedWorker isupervisor (int p))) (.assigned isupervisor (keys new-assignment)) (.put local-state LS-LOCAL-ASSIGNMENTS new-assignment) (reset! (:curr-assignment supervisor) new-assignment) ;; remove any downloaded code that's no longer assigned or active ;; important that this happens after setting the local assignment so that ;; synchronize-supervisor doesn't try to launch workers for which the ;; resources don't exist (doseq [storm-id downloaded-storm-ids] (when-not (assigned-storm-ids storm-id) (log-message "Removing code for storm id " storm-id) (rmr (supervisor-stormdist-root conf storm-id)) )) (.add processes-event-manager sync-processes) ))) ;; in local state, supervisor stores who its current assignments are ;; another thread launches events to restart any dead processes if necessary (defserverfn mk-supervisor [conf shared-context ^ISupervisor isupervisor] (log-message "Starting Supervisor with conf " conf) (.prepare isupervisor conf (supervisor-isupervisor-dir conf)) (FileUtils/cleanDirectory (File. (supervisor-tmp-dir conf))) (let [supervisor (supervisor-data conf shared-context isupervisor) [event-manager processes-event-manager :as managers] [(event/event-manager false) (event/event-manager false)] sync-processes (partial sync-processes supervisor) synchronize-supervisor (mk-synchronize-supervisor supervisor sync-processes event-manager processes-event-manager) heartbeat-fn (fn [] (.supervisor-heartbeat! (:storm-cluster-state supervisor) (:supervisor-id supervisor) (SupervisorInfo. (current-time-secs) (:my-hostname supervisor) (:assignment-id supervisor) (keys @(:curr-assignment supervisor)) ;; used ports (.getMetadata isupervisor) (conf SUPERVISOR-SCHEDULER-META) ((:uptime supervisor)))))] (heartbeat-fn) ;; should synchronize supervisor so it doesn't launch anything after being down (optimization) (schedule-recurring (:timer supervisor) 0 (conf SUPERVISOR-HEARTBEAT-FREQUENCY-SECS) heartbeat-fn) (when (conf SUPERVISOR-ENABLE) ;; This isn't strictly necessary, but it doesn't hurt and ensures that the machine stays up ;; to date even if callbacks don't all work exactly right (schedule-recurring (:timer supervisor) 0 10 (fn [] (.add event-manager synchronize-supervisor))) (schedule-recurring (:timer supervisor) 0 (conf SUPERVISOR-MONITOR-FREQUENCY-SECS) (fn [] (.add processes-event-manager sync-processes)))) (log-message "Starting supervisor with id " (:supervisor-id supervisor) " at host " (:my-hostname supervisor)) (reify Shutdownable (shutdown [this] (log-message "Shutting down supervisor " (:supervisor-id supervisor)) (reset! (:active supervisor) false) (cancel-timer (:timer supervisor)) (.shutdown event-manager) (.shutdown processes-event-manager) (.disconnect (:storm-cluster-state supervisor))) SupervisorDaemon (get-conf [this] conf) (get-id [this] (:supervisor-id supervisor)) (shutdown-all-workers [this] (let [ids (my-worker-ids conf)] (doseq [id ids] (shutdown-worker supervisor id) ))) DaemonCommon (waiting? [this] (or (not @(:active supervisor)) (and (timer-waiting? (:timer supervisor)) (every? (memfn waiting?) managers))) )))) (defn kill-supervisor [supervisor] (.shutdown supervisor) ) ;; distributed implementation (defmethod download-storm-code :distributed [conf storm-id master-code-dir] ;; Downloading to permanent location is atomic (let [tmproot (str (supervisor-tmp-dir conf) "/" (uuid)) stormroot (supervisor-stormdist-root conf storm-id)] (FileUtils/forceMkdir (File. tmproot)) (Utils/downloadFromMaster conf (master-stormjar-path master-code-dir) (supervisor-stormjar-path tmproot)) (Utils/downloadFromMaster conf (master-stormcode-path master-code-dir) (supervisor-stormcode-path tmproot)) (Utils/downloadFromMaster conf (master-stormconf-path master-code-dir) (supervisor-stormconf-path tmproot)) (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot) (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) )) (defmethod launch-worker :distributed [supervisor storm-id port worker-id] (let [conf (:conf supervisor) storm-home (System/getProperty "storm.home") stormroot (supervisor-stormdist-root conf storm-id) stormjar (supervisor-stormjar-path stormroot) storm-conf (read-supervisor-storm-conf conf storm-id) classpath (add-to-classpath (current-classpath) [stormjar]) childopts (.replaceAll (str (conf WORKER-CHILDOPTS) " " (storm-conf TOPOLOGY-WORKER-CHILDOPTS)) "%ID%" (str port)) logfilename (str "worker-" port ".log") command (str "java -server " childopts " -Djava.library.path=" (conf JAVA-LIBRARY-PATH) " -Dlogfile.name=" logfilename " -Dstorm.home=" storm-home " -Dlogback.configurationFile=" storm-home "/logback/cluster.xml" " -Dstorm.id=" storm-id " -Dworker.id=" worker-id " -Dworker.port=" port " -cp " classpath " backtype.storm.daemon.worker " (java.net.URLEncoder/encode storm-id) " " (:assignment-id supervisor) " " port " " worker-id)] (log-message "Launching worker with command: " command) (launch-process command :environment {"LD_LIBRARY_PATH" (conf JAVA-LIBRARY-PATH)}) )) ;; local implementation (defn resources-jar [] (->> (.split (current-classpath) File/pathSeparator) (filter #(.endsWith % ".jar")) (filter #(zip-contains-dir? % RESOURCES-SUBDIR)) first )) (defmethod download-storm-code :local [conf storm-id master-code-dir] (let [stormroot (supervisor-stormdist-root conf storm-id)] (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot)) (let [classloader (.getContextClassLoader (Thread/currentThread)) resources-jar (resources-jar) url (.getResource classloader RESOURCES-SUBDIR) target-dir (str stormroot "/" RESOURCES-SUBDIR)] (cond resources-jar (do (log-message "Extracting resources from jar at " resources-jar " to " target-dir) (extract-dir-from-jar resources-jar RESOURCES-SUBDIR stormroot)) url (do (log-message "Copying resources at " (str url) " to " target-dir) (FileUtils/copyDirectory (File. (.getFile url)) (File. target-dir)) )) ))) (defmethod launch-worker :local [supervisor storm-id port worker-id] (let [conf (:conf supervisor) pid (uuid) worker (worker/mk-worker conf (:shared-context supervisor) storm-id (:assignment-id supervisor) port worker-id)] (psim/register-process pid worker) (swap! (:worker-thread-pids-atom supervisor) assoc worker-id pid) )) (defn -launch [supervisor] (let [conf (read-storm-config)] (validate-distributed-mode! conf) (mk-supervisor conf nil supervisor))) (defn standalone-supervisor [] (let [conf-atom (atom nil) id-atom (atom nil)] (reify ISupervisor (prepare [this conf local-dir] (reset! conf-atom conf) (let [state (LocalState. local-dir) curr-id (if-let [id (.get state LS-ID)] id (generate-supervisor-id))] (.put state LS-ID curr-id) (reset! id-atom curr-id)) ) (confirmAssigned [this port] true) (getMetadata [this] (doall (map int (get @conf-atom SUPERVISOR-SLOTS-PORTS)))) (getSupervisorId [this] @id-atom) (getAssignmentId [this] @id-atom) (killedWorker [this port] ) (assigned [this ports] )))) (defn -main [] (-launch (standalone-supervisor))) ================================================ FILE: storm-core/src/clj/backtype/storm/daemon/task.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.task (:use [backtype.storm.daemon common]) (:use [backtype.storm bootstrap]) (:import [backtype.storm.hooks ITaskHook]) (:import [backtype.storm.tuple Tuple]) (:import [backtype.storm.generated SpoutSpec Bolt StateSpoutSpec]) (:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo EmitInfo BoltFailInfo BoltAckInfo]) (:require [backtype.storm [tuple :as tuple]]) (:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics])) (bootstrap) (defn mk-topology-context-builder [worker executor-data topology] (let [conf (:conf worker)] #(TopologyContext. topology (: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 (:storm-id worker))) (worker-pids-root conf (:worker-id worker)) (int %) (:port worker) (:task-ids worker) (:default-shared-resources worker) (:user-shared-resources worker) (:shared-executor-data executor-data) (:interval->task->metric-registry executor-data) (:open-or-prepare-was-called? executor-data)))) (defn system-topology-context [worker executor-data tid] ((mk-topology-context-builder worker executor-data (:system-topology worker)) tid)) (defn user-topology-context [worker executor-data tid] ((mk-topology-context-builder worker executor-data (:topology worker)) tid)) (defn- get-task-object [^TopologyContext topology component-id] (let [spouts (.get_spouts topology) bolts (.get_bolts topology) state-spouts (.get_state_spouts topology) obj (Utils/getSetComponentObject (cond (contains? spouts component-id) (.get_spout_object ^SpoutSpec (get spouts component-id)) (contains? bolts component-id) (.get_bolt_object ^Bolt (get bolts component-id)) (contains? state-spouts component-id) (.get_state_spout_object ^StateSpoutSpec (get state-spouts component-id)) true (throw-runtime "Could not find " component-id " in " topology))) obj (if (instance? ShellComponent obj) (if (contains? spouts component-id) (ShellSpout. obj) (ShellBolt. obj)) obj ) obj (if (instance? JavaObject obj) (thrift/instantiate-java-object obj) obj )] obj )) (defn get-context-hooks [^TopologyContext context] (.getHooks context)) (defn hooks-empty? [^Collection hooks] (.isEmpty hooks)) (defmacro apply-hooks [topology-context method-sym info-form] (let [hook-sym (with-meta (gensym "hook") {:tag 'backtype.storm.hooks.ITaskHook})] `(let [hooks# (get-context-hooks ~topology-context)] (when-not (hooks-empty? hooks#) (let [info# ~info-form] (fast-list-iter [~hook-sym hooks#] (~method-sym ~hook-sym info#) )))))) ;; TODO: this is all expensive... should be precomputed (defn send-unanchored ([task-data stream values overflow-buffer] (let [^TopologyContext topology-context (:system-context task-data) tasks-fn (:tasks-fn task-data) transfer-fn (-> task-data :executor-data :transfer-fn) out-tuple (TupleImpl. topology-context values (.getThisTaskId topology-context) stream)] (fast-list-iter [t (tasks-fn stream values)] (transfer-fn t out-tuple overflow-buffer) ))) ([task-data stream values] (send-unanchored task-data stream values nil) )) (defn mk-tasks-fn [task-data] (let [task-id (:task-id task-data) executor-data (:executor-data task-data) component-id (:component-id executor-data) ^WorkerTopologyContext worker-context (:worker-context executor-data) storm-conf (:storm-conf executor-data) emit-sampler (mk-stats-sampler storm-conf) stream->component->grouper (:stream->component->grouper executor-data) user-context (:user-context task-data) executor-stats (:stats executor-data) debug? (= true (storm-conf TOPOLOGY-DEBUG))] (fn ([^Integer out-task-id ^String stream ^List values] (when debug? (log-message "Emitting direct: " out-task-id "; " component-id " " stream " " values)) (let [target-component (.getComponentId worker-context out-task-id) component->grouping (get stream->component->grouper stream) grouping (get component->grouping target-component) out-task-id (if grouping out-task-id)] (when (and (not-nil? grouping) (not= :direct grouping)) (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping"))) (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id])) (when (emit-sampler) (builtin-metrics/emitted-tuple! (:builtin-metrics task-data) executor-stats stream) (stats/emitted-tuple! executor-stats stream) (if out-task-id (stats/transferred-tuples! executor-stats stream 1) (builtin-metrics/transferred-tuple! (:builtin-metrics task-data) executor-stats stream 1))) (if out-task-id [out-task-id]) )) ([^String stream ^List values] (when debug? (log-message "Emitting: " component-id " " stream " " values)) (let [out-tasks (ArrayList.)] (fast-map-iter [[out-component grouper] (get stream->component->grouper stream)] (when (= :direct grouper) ;; TODO: this is wrong, need to check how the stream was declared (throw (IllegalArgumentException. "Cannot do regular emit to direct stream"))) (let [comp-tasks (grouper task-id values)] (if (or (sequential? comp-tasks) (instance? Collection comp-tasks)) (.addAll out-tasks comp-tasks) (.add out-tasks comp-tasks) ))) (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks)) (when (emit-sampler) (stats/emitted-tuple! executor-stats stream) (builtin-metrics/emitted-tuple! (:builtin-metrics task-data) executor-stats stream) (stats/transferred-tuples! executor-stats stream (count out-tasks)) (builtin-metrics/transferred-tuple! (:builtin-metrics task-data) executor-stats stream (count out-tasks))) out-tasks))) )) (defn mk-task-data [executor-data task-id] (recursive-map :executor-data executor-data :task-id task-id :system-context (system-topology-context (:worker executor-data) executor-data task-id) :user-context (user-topology-context (:worker executor-data) executor-data task-id) :builtin-metrics (builtin-metrics/make-data (:type executor-data)) :tasks-fn (mk-tasks-fn <>) :object (get-task-object (.getRawTopology ^TopologyContext (:system-context <>)) (:component-id executor-data)))) (defn mk-task [executor-data task-id] (let [task-data (mk-task-data executor-data task-id) storm-conf (:storm-conf executor-data)] (doseq [klass (storm-conf TOPOLOGY-AUTO-TASK-HOOKS)] (.addTaskHook ^TopologyContext (:user-context task-data) (-> klass Class/forName .newInstance))) ;; when this is called, the threads for the executor haven't been started yet, ;; so we won't be risking trampling on the single-threaded claim strategy disruptor queue (send-unanchored task-data SYSTEM-STREAM-ID ["startup"]) task-data )) ================================================ FILE: storm-core/src/clj/backtype/storm/daemon/worker.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.worker (:use [backtype.storm.daemon common]) (:use [backtype.storm bootstrap]) (:require [backtype.storm.daemon [executor :as executor]]) (:import [java.util.concurrent Executors]) (:import [backtype.storm.messaging TransportFactory]) (:import [backtype.storm.messaging IContext IConnection]) (:gen-class)) (bootstrap) (defmulti mk-suicide-fn cluster-mode) (defn read-worker-executors [storm-conf storm-cluster-state storm-id assignment-id port] (let [assignment (:executor->node+port (.assignment-info storm-cluster-state storm-id nil))] (doall (concat [Constants/SYSTEM_EXECUTOR_ID] (mapcat (fn [[executor loc]] (if (= loc [assignment-id port]) [executor] )) assignment))))) (defnk do-executor-heartbeats [worker :executors nil] ;; stats is how we know what executors are assigned to this worker (let [stats (if-not executors (into {} (map (fn [e] {e nil}) (:executors worker))) (->> executors (map (fn [e] {(executor/get-executor-id e) (executor/render-stats e)})) (apply merge))) zk-hb {:storm-id (:storm-id worker) :executor-stats stats :uptime ((:uptime worker)) :time-secs (current-time-secs) }] ;; do the zookeeper heartbeat (.worker-heartbeat! (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) zk-hb) )) (defn do-heartbeat [worker] (let [conf (:conf worker) hb (WorkerHeartbeat. (current-time-secs) (:storm-id worker) (:executors worker) (:port worker)) state (worker-state conf (:worker-id worker))] (log-debug "Doing heartbeat " (pr-str hb)) ;; do the local-file-system heartbeat. (.put state LS-WORKER-HEARTBEAT hb false ) (.cleanup state 60) ; this is just in case supervisor is down so that disk doesn't fill up. ; it shouldn't take supervisor 120 seconds between listing dir and reading it )) (defn worker-outbound-tasks "Returns seq of task-ids that receive messages from this worker" [worker] (let [context (worker-context worker) components (mapcat (fn [task-id] (->> (.getComponentId context (int task-id)) (.getTargets context) vals (map keys) (apply concat))) (:task-ids worker))] (-> worker :task->component reverse-map (select-keys components) vals flatten set ))) (defn mk-transfer-local-fn [worker] (let [short-executor-receive-queue-map (:short-executor-receive-queue-map worker) task->short-executor (:task->short-executor worker) task-getter (comp #(get task->short-executor %) fast-first)] (fn [tuple-batch] (let [grouped (fast-group-by task-getter tuple-batch)] (fast-map-iter [[short-executor pairs] grouped] (let [q (short-executor-receive-queue-map short-executor)] (if q (disruptor/publish q pairs) (log-warn "Received invalid messages for unknown tasks. Dropping... ") ))))))) (defn mk-transfer-fn [worker] (let [local-tasks (-> worker :task-ids set) local-transfer (:transfer-local-fn worker) ^DisruptorQueue transfer-queue (:transfer-queue worker)] (fn [^KryoTupleSerializer serializer tuple-batch] (let [local (ArrayList.) remote (ArrayList.)] (fast-list-iter [[task tuple :as pair] tuple-batch] (if (local-tasks task) (.add local pair) (.add remote pair) )) (local-transfer local) ;; not using map because the lazy seq shows up in perf profiles (let [serialized-pairs (fast-list-for [[task ^TupleImpl tuple] remote] [task (.serialize serializer tuple)])] (disruptor/publish transfer-queue serialized-pairs) ))))) (defn- mk-receive-queue-map [storm-conf executors] (->> executors ;; TODO: this depends on the type of executor (map (fn [e] [e (disruptor/disruptor-queue (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE) :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))])) (into {}) )) (defn- stream->fields [^StormTopology topology component] (->> (ThriftTopologyUtils/getComponentCommon topology component) .get_streams (map (fn [[s info]] [s (Fields. (.get_output_fields info))])) (into {}) (HashMap.))) (defn component->stream->fields [^StormTopology topology] (->> (ThriftTopologyUtils/getComponentIds topology) (map (fn [c] [c (stream->fields topology c)])) (into {}) (HashMap.))) (defn- mk-default-resources [worker] (let [conf (:conf worker) thread-pool-size (int (conf TOPOLOGY-WORKER-SHARED-THREAD-POOL-SIZE))] {WorkerTopologyContext/SHARED_EXECUTOR (Executors/newFixedThreadPool thread-pool-size)} )) (defn- mk-user-resources [worker] ;;TODO: need to invoke a hook provided by the topology, giving it a chance to create user resources. ;; this would be part of the initialization hook ;; need to separate workertopologycontext into WorkerContext and WorkerUserContext. ;; actually just do it via interfaces. just need to make sure to hide setResource from tasks {}) (defn mk-halting-timer [] (mk-timer :kill-fn (fn [t] (log-error t "Error when processing event") (halt-process! 20 "Error when processing an event") ))) (defn worker-data [conf mq-context storm-id assignment-id port worker-id] (let [cluster-state (cluster/mk-distributed-cluster-state conf) storm-cluster-state (cluster/mk-storm-cluster-state cluster-state) storm-conf (read-supervisor-storm-conf conf storm-id) executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port)) transfer-queue (disruptor/disruptor-queue (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE) :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY)) executor-receive-queue-map (mk-receive-queue-map storm-conf executors) receive-queue-map (->> executor-receive-queue-map (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue]))) (into {})) topology (read-supervisor-topology conf storm-id)] (recursive-map :conf conf :mq-context (if mq-context mq-context (TransportFactory/makeContext storm-conf)) :storm-id storm-id :assignment-id assignment-id :port port :worker-id worker-id :cluster-state cluster-state :storm-cluster-state storm-cluster-state :storm-active-atom (atom false) :executors executors :task-ids (->> receive-queue-map keys (map int) sort) :storm-conf storm-conf :topology topology :system-topology (system-topology! storm-conf topology) :heartbeat-timer (mk-halting-timer) :refresh-connections-timer (mk-halting-timer) :refresh-active-timer (mk-halting-timer) :executor-heartbeat-timer (mk-halting-timer) :user-timer (mk-halting-timer) :task->component (HashMap. (storm-task-info topology storm-conf)) ; for optimized access when used in tasks later on :component->stream->fields (component->stream->fields (:system-topology <>)) :component->sorted-tasks (->> (:task->component <>) reverse-map (map-val sort)) :endpoint-socket-lock (mk-rw-lock) :cached-node+port->socket (atom {}) :cached-task->node+port (atom {}) :transfer-queue transfer-queue :executor-receive-queue-map executor-receive-queue-map :short-executor-receive-queue-map (map-key first executor-receive-queue-map) :task->short-executor (->> executors (mapcat (fn [e] (for [t (executor-id->tasks e)] [t (first e)]))) (into {}) (HashMap.)) :suicide-fn (mk-suicide-fn conf) :uptime (uptime-computer) :default-shared-resources (mk-default-resources <>) :user-shared-resources (mk-user-resources <>) :transfer-local-fn (mk-transfer-local-fn <>) :transfer-fn (mk-transfer-fn <>) ))) (defn- endpoint->string [[node port]] (str port "/" node)) (defn string->endpoint [^String s] (let [[port-str node] (.split s "/" 2)] [node (Integer/valueOf port-str)] )) (defn mk-refresh-connections [worker] (let [outbound-tasks (worker-outbound-tasks worker) conf (:conf worker) storm-cluster-state (:storm-cluster-state worker) storm-id (:storm-id worker)] (fn this ([] (this (fn [& ignored] (schedule (:refresh-connections-timer worker) 0 this)))) ([callback] (let [assignment (.assignment-info storm-cluster-state storm-id callback) my-assignment (-> assignment :executor->node+port to-task->node+port (select-keys outbound-tasks) (#(map-val endpoint->string %))) ;; we dont need a connection for the local tasks anymore needed-assignment (->> my-assignment (filter-key (complement (-> worker :task-ids set)))) needed-connections (-> needed-assignment vals set) needed-tasks (-> needed-assignment keys) current-connections (set (keys @(:cached-node+port->socket worker))) new-connections (set/difference needed-connections current-connections) remove-connections (set/difference current-connections needed-connections)] (swap! (:cached-node+port->socket worker) #(HashMap. (merge (into {} %1) %2)) (into {} (dofor [endpoint-str new-connections :let [[node port] (string->endpoint endpoint-str)]] [endpoint-str (.connect ^IContext (:mq-context worker) storm-id ((:node->host assignment) node) port) ] ))) (write-locked (:endpoint-socket-lock worker) (reset! (:cached-task->node+port worker) (HashMap. my-assignment))) (doseq [endpoint remove-connections] (.close (get @(:cached-node+port->socket worker) endpoint))) (apply swap! (:cached-node+port->socket worker) #(HashMap. (apply dissoc (into {} %1) %&)) remove-connections) (let [missing-tasks (->> needed-tasks (filter (complement my-assignment)))] (when-not (empty? missing-tasks) (log-warn "Missing assignment for following tasks: " (pr-str missing-tasks)) ))))))) (defn refresh-storm-active ([worker] (refresh-storm-active worker (fn [& ignored] (schedule (:refresh-active-timer worker) 0 (partial refresh-storm-active worker))))) ([worker callback] (let [base (.storm-base (:storm-cluster-state worker) (:storm-id worker) callback)] (reset! (:storm-active-atom worker) (= :active (-> base :status :type)) )) )) ;; TODO: consider having a max batch size besides what disruptor does automagically to prevent latency issues (defn mk-transfer-tuples-handler [worker] (let [^DisruptorQueue transfer-queue (:transfer-queue worker) drainer (ArrayList.) node+port->socket (:cached-node+port->socket worker) task->node+port (:cached-task->node+port worker) endpoint-socket-lock (:endpoint-socket-lock worker) ] (disruptor/clojure-handler (fn [packets _ batch-end?] (.addAll drainer packets) (when batch-end? (read-locked endpoint-socket-lock (let [node+port->socket @node+port->socket task->node+port @task->node+port] ;; consider doing some automatic batching here (would need to not be serialized at this point to remove per-tuple overhead) ;; try using multipart messages ... first sort the tuples by the target node (without changing the local ordering) (fast-list-iter [[task ser-tuple] drainer] ;; TODO: consider write a batch of tuples here to every target worker ;; group by node+port, do multipart send (let [node-port (get task->node+port task)] (when node-port (.send ^IConnection (get node+port->socket node-port) task ser-tuple)) )))) (.clear drainer)))))) (defn launch-receive-thread [worker] (log-message "Launching receive-thread for " (:assignment-id worker) ":" (:port worker)) (msg-loader/launch-receive-thread! (:mq-context worker) (:storm-id worker) (:port worker) (:transfer-local-fn worker) (-> worker :storm-conf (get TOPOLOGY-RECEIVER-BUFFER-SIZE)) :kill-fn (fn [t] (halt-process! 11)))) (defn- close-resources [worker] (let [dr (:default-shared-resources worker)] (log-message "Shutting down default resources") (.shutdownNow (get dr WorkerTopologyContext/SHARED_EXECUTOR)) (log-message "Shut down default resources"))) ;; TODO: should worker even take the storm-id as input? this should be ;; deducable from cluster state (by searching through assignments) ;; what about if there's inconsistency in assignments? -> but nimbus ;; should guarantee this consistency ;; TODO: consider doing worker heartbeating rather than task heartbeating to reduce the load on zookeeper (defserverfn mk-worker [conf shared-mq-context storm-id assignment-id port worker-id] (log-message "Launching worker for " storm-id " on " assignment-id ":" port " with id " worker-id " and conf " conf) (if-not (local-mode? conf) (redirect-stdio-to-slf4j!)) ;; because in local mode, its not a separate ;; process. supervisor will register it in this case (when (= :distributed (cluster-mode conf)) (touch (worker-pid-path conf worker-id (process-pid)))) (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id) heartbeat-fn #(do-heartbeat worker) ;; do this here so that the worker process dies if this fails ;; it's important that worker heartbeat to supervisor ASAP when launching so that the supervisor knows it's running (and can move on) _ (heartbeat-fn) ;; heartbeat immediately to nimbus so that it knows that the worker has been started _ (do-executor-heartbeats worker) executors (atom nil) ;; launch heartbeat threads immediately so that slow-loading tasks don't cause the worker to timeout ;; to the supervisor _ (schedule-recurring (:heartbeat-timer worker) 0 (conf WORKER-HEARTBEAT-FREQUENCY-SECS) heartbeat-fn) _ (schedule-recurring (:executor-heartbeat-timer worker) 0 (conf TASK-HEARTBEAT-FREQUENCY-SECS) #(do-executor-heartbeats worker :executors @executors)) refresh-connections (mk-refresh-connections worker) _ (refresh-connections nil) _ (refresh-storm-active worker nil) _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e))) receive-thread-shutdown (launch-receive-thread worker) transfer-tuples (mk-transfer-tuples-handler worker) transfer-thread (disruptor/consume-loop* (:transfer-queue worker) transfer-tuples) shutdown* (fn [] (log-message "Shutting down worker " storm-id " " assignment-id " " port) (doseq [[_ socket] @(:cached-node+port->socket worker)] ;; this will do best effort flushing since the linger period ;; was set on creation (.close socket)) (log-message "Shutting down receive thread") (receive-thread-shutdown) (log-message "Shut down receive thread") (log-message "Terminating messaging context") (log-message "Shutting down executors") (doseq [executor @executors] (.shutdown executor)) (log-message "Shut down executors") ;;this is fine because the only time this is shared is when it's a local context, ;;in which case it's a noop (.term ^IContext (:mq-context worker)) (log-message "Shutting down transfer thread") (disruptor/halt-with-interrupt! (:transfer-queue worker)) (.interrupt transfer-thread) (.join transfer-thread) (log-message "Shut down transfer thread") (cancel-timer (:heartbeat-timer worker)) (cancel-timer (:refresh-connections-timer worker)) (cancel-timer (:refresh-active-timer worker)) (cancel-timer (:executor-heartbeat-timer worker)) (cancel-timer (:user-timer worker)) (close-resources worker) ;; TODO: here need to invoke the "shutdown" method of WorkerHook (.remove-worker-heartbeat! (:storm-cluster-state worker) storm-id assignment-id port) (log-message "Disconnecting from storm cluster state context") (.disconnect (:storm-cluster-state worker)) (.close (:cluster-state worker)) (log-message "Shut down worker " storm-id " " assignment-id " " port)) ret (reify Shutdownable (shutdown [this] (shutdown*)) DaemonCommon (waiting? [this] (and (timer-waiting? (:heartbeat-timer worker)) (timer-waiting? (:refresh-connections-timer worker)) (timer-waiting? (:refresh-active-timer worker)) (timer-waiting? (:executor-heartbeat-timer worker)) (timer-waiting? (:user-timer worker)) )) )] (schedule-recurring (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections) (schedule-recurring (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker)) (log-message "Worker has topology config " (:storm-conf worker)) (log-message "Worker " worker-id " for storm " storm-id " on " assignment-id ":" port " has finished loading") ret )) (defmethod mk-suicide-fn :local [conf] (fn [] (halt-process! 1 "Worker died"))) (defmethod mk-suicide-fn :distributed [conf] (fn [] (halt-process! 1 "Worker died"))) (defn -main [storm-id assignment-id port-str worker-id] (let [conf (read-storm-config)] (validate-distributed-mode! conf) (mk-worker conf nil (java.net.URLDecoder/decode storm-id) assignment-id (Integer/parseInt port-str) worker-id))) ================================================ FILE: storm-core/src/clj/backtype/storm/disruptor.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.disruptor (:import [backtype.storm.utils DisruptorQueue]) (:import [com.lmax.disruptor MultiThreadedClaimStrategy SingleThreadedClaimStrategy BlockingWaitStrategy SleepingWaitStrategy YieldingWaitStrategy BusySpinWaitStrategy]) (:require [clojure [string :as str]]) (:require [clojure [set :as set]]) (:use [clojure walk]) (:use [backtype.storm util log]) ) (def CLAIM-STRATEGY {:multi-threaded (fn [size] (MultiThreadedClaimStrategy. (int size))) :single-threaded (fn [size] (SingleThreadedClaimStrategy. (int size))) }) (def WAIT-STRATEGY {:block (fn [] (BlockingWaitStrategy.)) :yield (fn [] (YieldingWaitStrategy.)) :sleep (fn [] (SleepingWaitStrategy.)) :spin (fn [] (BusySpinWaitStrategy.)) }) (defn- mk-wait-strategy [spec] (if (keyword? spec) ((WAIT-STRATEGY spec)) (-> (str spec) new-instance) )) ;; :block strategy requires using a timeout on waitFor (implemented in DisruptorQueue), as sometimes the consumer stays blocked even when there's an item on the queue. ;; This would manifest itself in Trident when doing 1 batch at a time processing, and the ack_init message ;; wouldn't make it to the acker until the batch timed out and another tuple was played into the queue, ;; unblocking the consumer (defnk disruptor-queue [buffer-size :claim-strategy :multi-threaded :wait-strategy :block] (DisruptorQueue. ((CLAIM-STRATEGY claim-strategy) buffer-size) (mk-wait-strategy wait-strategy) )) (defn clojure-handler [afn] (reify com.lmax.disruptor.EventHandler (onEvent [this o seq-id batchEnd?] (afn o seq-id batchEnd?) ))) (defmacro handler [& args] `(clojure-handler (fn ~@args))) (defn publish ([^DisruptorQueue q o block?] (.publish q o block?)) ([q o] (publish q o true))) (defn try-publish [^DisruptorQueue q o] (.tryPublish q o)) (defn consume-batch [^DisruptorQueue queue handler] (.consumeBatch queue handler)) (defn consume-batch-when-available [^DisruptorQueue queue handler] (.consumeBatchWhenAvailable queue handler)) (defn consumer-started! [^DisruptorQueue queue] (.consumerStarted queue)) (defn halt-with-interrupt! [^DisruptorQueue queue] (.haltWithInterrupt queue)) (defnk consume-loop* [^DisruptorQueue queue handler :kill-fn (fn [error] (halt-process! 1 "Async loop died!")) :thread-name nil] (let [ret (async-loop (fn [] (consume-batch-when-available queue handler) 0 ) :kill-fn kill-fn :thread-name thread-name )] (consumer-started! queue) ret )) (defmacro consume-loop [queue & handler-args] `(let [handler# (handler ~@handler-args)] (consume-loop* ~queue handler#) )) ================================================ FILE: storm-core/src/clj/backtype/storm/event.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.event (:use [backtype.storm log util]) (:import [backtype.storm.utils Time Utils]) (:import [java.util.concurrent LinkedBlockingQueue TimeUnit]) ) (defprotocol EventManager (add [this event-fn]) (waiting? [this]) (shutdown [this])) (defn event-manager "Creates a thread to respond to events. Any error will cause process to halt" [daemon?] (let [added (atom 0) processed (atom 0) ^LinkedBlockingQueue queue (LinkedBlockingQueue.) running (atom true) runner (Thread. (fn [] (try-cause (while @running (let [r (.take queue)] (r) (swap! processed inc))) (catch InterruptedException t (log-message "Event manager interrupted")) (catch Throwable t (log-error t "Error when processing event") (halt-process! 20 "Error when processing an event")) )))] (.setDaemon runner daemon?) (.start runner) (reify EventManager (add [this event-fn] ;; should keep track of total added and processed to know if this is finished yet (when-not @running (throw (RuntimeException. "Cannot add events to a shutdown event manager"))) (swap! added inc) (.put queue event-fn) ) (waiting? [this] (or (Time/isThreadWaiting runner) (= @processed @added) )) (shutdown [this] (reset! running false) (.interrupt runner) (.join runner) ) ))) ================================================ FILE: storm-core/src/clj/backtype/storm/log.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.log (:require [clojure.tools [logging :as log]])) (defmacro log-message [& args] `(log/info (str ~@args))) (defmacro log-error [e & args] `(log/log :error ~e (str ~@args))) (defmacro log-debug [& args] `(log/debug (str ~@args))) (defmacro log-warn-error [e & args] `(log/warn (str ~@args) ~e)) (defmacro log-warn [& args] `(log/warn (str ~@args))) (defn log-capture! [& args] (apply log/log-capture! args)) (defn log-stream [& args] (apply log/log-stream args)) ================================================ FILE: storm-core/src/clj/backtype/storm/messaging/loader.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.messaging.loader (:use [backtype.storm util log]) (:import [java.util ArrayList]) (:import [backtype.storm.messaging IContext IConnection TaskMessage]) (:import [backtype.storm.utils DisruptorQueue MutableObject]) (:require [backtype.storm.messaging [local :as local]]) (:require [backtype.storm [disruptor :as disruptor]])) (defn mk-local-context [] (local/mk-context)) (defnk launch-receive-thread! [context storm-id port transfer-local-fn max-buffer-size :daemon true :kill-fn (fn [t] (System/exit 1)) :priority Thread/NORM_PRIORITY] (let [max-buffer-size (int max-buffer-size) vthread (async-loop (fn [] (let [socket (.bind ^IContext context storm-id port)] (fn [] (let [batched (ArrayList.) init (.recv ^IConnection socket 0)] (loop [packet init] (let [task (if packet (.task ^TaskMessage packet)) message (if packet (.message ^TaskMessage packet))] (if (= task -1) (do (log-message "Receiving-thread:[" storm-id ", " port "] received shutdown notice") (.close socket) nil ) (do (when packet (.add batched [task message])) (if (and packet (< (.size batched) max-buffer-size)) (recur (.recv ^IConnection socket 1)) (do (transfer-local-fn batched) 0 )))))))))) :factory? true :daemon daemon :kill-fn kill-fn :priority priority)] (fn [] (let [kill-socket (.connect ^IContext context storm-id "localhost" port)] (log-message "Shutting down receiving-thread: [" storm-id ", " port "]") (.send ^IConnection kill-socket -1 (byte-array [])) (log-message "Waiting for receiving-thread:[" storm-id ", " port "] to die") (.join vthread) (.close ^IConnection kill-socket) (log-message "Shutdown receiving-thread: [" storm-id ", " port "]") )))) ================================================ FILE: storm-core/src/clj/backtype/storm/messaging/local.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.messaging.local (:refer-clojure :exclude [send]) (:use [backtype.storm log]) (:import [backtype.storm.messaging IContext IConnection TaskMessage]) (:import [java.util.concurrent LinkedBlockingQueue]) (:import [java.util Map]) (:gen-class)) (defn add-queue! [queues-map lock storm-id port] (let [id (str storm-id "-" port)] (locking lock (when-not (contains? @queues-map id) (swap! queues-map assoc id (LinkedBlockingQueue.)))) (@queues-map id))) (deftype LocalConnection [storm-id port queues-map lock queue] IConnection (^TaskMessage recv [this ^int flags] (when-not queue (throw (IllegalArgumentException. "Cannot receive on this socket"))) (if (= flags 1) (.poll queue) (.take queue))) (^void send [this ^int taskId ^bytes payload] (let [send-queue (add-queue! queues-map lock storm-id port)] (.put send-queue (TaskMessage. taskId payload)) )) (^void close [this] )) (deftype LocalContext [^{:unsynchronized-mutable true} queues-map ^{:unsynchronized-mutable true} lock] IContext (^void prepare [this ^Map storm-conf] (set! queues-map (atom {})) (set! lock (Object.))) (^IConnection bind [this ^String storm-id ^int port] (LocalConnection. storm-id port queues-map lock (add-queue! queues-map lock storm-id port))) (^IConnection connect [this ^String storm-id ^String host ^int port] (LocalConnection. storm-id port queues-map lock nil)) (^void term [this] )) (defn mk-context [] (let [context (LocalContext. nil nil)] (.prepare ^IContext context nil) context)) ================================================ FILE: storm-core/src/clj/backtype/storm/messaging/zmq.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.messaging.zmq (:refer-clojure :exclude [send]) (:use [backtype.storm config log]) (:import [backtype.storm.messaging IContext IConnection TaskMessage]) (:import [java.nio ByteBuffer]) (:import [org.zeromq ZMQ]) (:import [java.util Map]) (:require [zilch.mq :as mq]) (:gen-class :methods [^{:static true} [makeContext [java.util.Map] backtype.storm.messaging.IContext]])) (defn mk-packet [task ^bytes message] (let [bb (ByteBuffer/allocate (+ 2 (count message)))] (.putShort bb (short task)) (.put bb message) (.array bb) )) (defn parse-packet [^bytes packet] (let [bb (ByteBuffer/wrap packet) port (.getShort bb) msg (byte-array (- (count packet) 2))] (.get bb msg) (TaskMessage. (int port) msg) )) (defn get-bind-zmq-url [local? port] (if local? (str "ipc://" port ".ipc") (str "tcp://*:" port))) (defn get-connect-zmq-url [local? host port] (if local? (str "ipc://" port ".ipc") (str "tcp://" host ":" port))) (defprotocol ZMQContextQuery (zmq-context [this])) (deftype ZMQConnection [socket] IConnection (^TaskMessage recv [this ^int flags] (require 'backtype.storm.messaging.zmq) (if-let [packet (mq/recv socket flags)] (parse-packet packet))) (^void send [this ^int taskId ^bytes payload] (require 'backtype.storm.messaging.zmq) (mq/send socket (mk-packet taskId payload) ZMQ/NOBLOCK)) ;; TODO: how to do backpressure if doing noblock?... need to only unblock if the target disappears (^void close [this] (.close socket))) (defn mk-connection [socket] (ZMQConnection. socket)) (deftype ZMQContext [^{:unsynchronized-mutable true} context ^{:unsynchronized-mutable true} linger-ms ^{:unsynchronized-mutable true} hwm ^{:unsynchronized-mutable true} local?] IContext (^void prepare [this ^Map storm-conf] (let [num-threads (.get storm-conf ZMQ-THREADS)] (set! context (mq/context num-threads)) (set! linger-ms (.get storm-conf ZMQ-LINGER-MILLIS)) (set! hwm (.get storm-conf ZMQ-HWM)) (set! local? (= (.get storm-conf STORM-CLUSTER-MODE) "local")))) (^IConnection bind [this ^String storm-id ^int port] (require 'backtype.storm.messaging.zmq) (-> context (mq/socket mq/pull) (mq/set-hwm hwm) (mq/bind (get-bind-zmq-url local? port)) mk-connection )) (^IConnection connect [this ^String storm-id ^String host ^int port] (require 'backtype.storm.messaging.zmq) (-> context (mq/socket mq/push) (mq/set-hwm hwm) (mq/set-linger linger-ms) (mq/connect (get-connect-zmq-url local? host port)) mk-connection)) (^void term [this] (.term context)) ZMQContextQuery (zmq-context [this] context)) (defn -makeContext [^Map storm-conf] (let [context (ZMQContext. nil 0 0 true)] (.prepare context storm-conf) context)) ================================================ FILE: storm-core/src/clj/backtype/storm/metric/testing.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.metric.testing "This namespace is for AOT dependent metrics testing code." (:gen-class)) (letfn [(for- [threader arg seq-exprs body] `(reduce #(%2 %1) ~arg (for ~seq-exprs (fn [arg#] (~threader arg# ~@body)))))] (defmacro for-> "Apply a thread expression to a sequence. eg. (-> 1 (for-> [x [1 2 3]] (+ x))) => 7" {:indent 1} [arg seq-exprs & body] (for- 'clojure.core/-> arg seq-exprs body))) (gen-class :name clojure.storm.metric.testing.FakeMetricConsumer :implements [backtype.storm.metric.api.IMetricsConsumer] :prefix "impl-") (def buffer (atom nil)) (defn impl-prepare [this conf argument ctx error-reporter] (reset! buffer {})) (defn impl-cleanup [this] (reset! buffer {})) (defn vec-conj [coll x] (if coll (conj coll x) [x])) (defn expand-complex-datapoint [dp] (if (or (map? (.value dp)) (instance? java.util.AbstractMap (.value dp))) (into [] (for [[k v] (.value dp)] [(str (.name dp) "/" k) v])) [[(.name dp) (.value dp)]])) (defn impl-handleDataPoints [this task-info data-points] (swap! buffer (fn [old] (-> old (for-> [dp data-points [name val] (expand-complex-datapoint dp)] (update-in [(.srcComponentId task-info) name (.srcTaskId task-info)] vec-conj val)))))) ================================================ FILE: storm-core/src/clj/backtype/storm/process_simulator.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.process-simulator (:use [backtype.storm log util]) ) (def pid-counter (mk-counter)) (def process-map (atom {})) (def kill-lock (Object.)) (defn register-process [pid shutdownable] (swap! process-map assoc pid shutdownable)) (defn process-handle [pid] (@process-map pid)) (defn all-processes [] (vals @process-map)) (defn kill-process [pid] (locking kill-lock ; in case cluster shuts down while supervisor is ; killing a task (log-message "Killing process " pid) (let [shutdownable (process-handle pid)] (swap! process-map dissoc pid) (when shutdownable (.shutdown shutdownable)) ))) (defn kill-all-processes [] (doseq [pid (keys @process-map)] (kill-process pid) )) ================================================ FILE: storm-core/src/clj/backtype/storm/scheduler/DefaultScheduler.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.scheduler.DefaultScheduler (:use [backtype.storm util config]) (:require [backtype.storm.scheduler.EvenScheduler :as EvenScheduler]) (:import [backtype.storm.scheduler IScheduler Topologies Cluster TopologyDetails WorkerSlot SchedulerAssignment EvenScheduler ExecutorDetails]) (:gen-class :implements [backtype.storm.scheduler.IScheduler])) (defn- bad-slots [existing-slots num-executors num-workers] (if (= 0 num-workers) '() (let [distribution (atom (integer-divided num-executors num-workers)) keepers (atom {})] (doseq [[node+port executor-list] existing-slots :let [executor-count (count executor-list)]] (when (pos? (get @distribution executor-count 0)) (swap! keepers assoc node+port executor-list) (swap! distribution update-in [executor-count] dec) )) (->> @keepers keys (apply dissoc existing-slots) keys (map (fn [[node port]] (WorkerSlot. node port))))))) (defn slots-can-reassign [^Cluster cluster slots] (->> slots (filter (fn [[node port]] (if-not (.isBlackListed cluster node) (if-let [supervisor (.getSupervisorById cluster node)] (.contains (.getAllPorts supervisor) (int port)) )))))) (defn -prepare [this conf] ) (defn default-schedule [^Topologies topologies ^Cluster cluster] (let [needs-scheduling-topologies (.needsSchedulingTopologies cluster topologies)] (doseq [^TopologyDetails topology needs-scheduling-topologies :let [topology-id (.getId topology) available-slots (->> (.getAvailableSlots cluster) (map #(vector (.getNodeId %) (.getPort %)))) all-executors (->> topology .getExecutors (map #(vector (.getStartTask %) (.getEndTask %))) set) alive-assigned (EvenScheduler/get-alive-assigned-node+port->executors cluster topology-id) alive-executors (->> alive-assigned vals (apply concat) set) can-reassign-slots (slots-can-reassign cluster (keys alive-assigned)) total-slots-to-use (min (.getNumWorkers topology) (+ (count can-reassign-slots) (count available-slots))) bad-slots (if (or (> total-slots-to-use (count alive-assigned)) (not= alive-executors all-executors)) (bad-slots alive-assigned (count all-executors) total-slots-to-use) [])]] (.freeSlots cluster bad-slots) (EvenScheduler/schedule-topologies-evenly (Topologies. {topology-id topology}) cluster)))) (defn -schedule [this ^Topologies topologies ^Cluster cluster] (default-schedule topologies cluster)) ================================================ FILE: storm-core/src/clj/backtype/storm/scheduler/EvenScheduler.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.scheduler.EvenScheduler (:use [backtype.storm util log config]) (:require [clojure.set :as set]) (:import [backtype.storm.scheduler IScheduler Topologies Cluster TopologyDetails WorkerSlot ExecutorDetails]) (:gen-class :implements [backtype.storm.scheduler.IScheduler])) (defn sort-slots [all-slots] (let [split-up (vals (group-by first all-slots))] (apply interleave-all split-up) )) (defn get-alive-assigned-node+port->executors [cluster topology-id] (let [existing-assignment (.getAssignmentById cluster topology-id) executor->slot (if existing-assignment (.getExecutorToSlot existing-assignment) {}) executor->node+port (into {} (for [[^ExecutorDetails executor ^WorkerSlot slot] executor->slot :let [executor [(.getStartTask executor) (.getEndTask executor)] node+port [(.getNodeId slot) (.getPort slot)]]] {executor node+port})) alive-assigned (reverse-map executor->node+port)] alive-assigned)) (defn- schedule-topology [^TopologyDetails topology ^Cluster cluster] (let [topology-id (.getId topology) available-slots (->> (.getAvailableSlots cluster) (map #(vector (.getNodeId %) (.getPort %)))) all-executors (->> topology .getExecutors (map #(vector (.getStartTask %) (.getEndTask %))) set) alive-assigned (get-alive-assigned-node+port->executors cluster topology-id) total-slots-to-use (min (.getNumWorkers topology) (+ (count available-slots) (count alive-assigned))) reassign-slots (take (- total-slots-to-use (count alive-assigned)) (sort-slots available-slots)) reassign-executors (sort (set/difference all-executors (set (apply concat (vals alive-assigned))))) reassignment (into {} (map vector reassign-executors ;; for some reason it goes into infinite loop without limiting the repeat-seq (repeat-seq (count reassign-executors) reassign-slots)))] (when-not (empty? reassignment) (log-message "Available slots: " (pr-str available-slots)) ) reassignment)) (defn schedule-topologies-evenly [^Topologies topologies ^Cluster cluster] (let [needs-scheduling-topologies (.needsSchedulingTopologies cluster topologies)] (doseq [^TopologyDetails topology needs-scheduling-topologies :let [topology-id (.getId topology) new-assignment (schedule-topology topology cluster) node+port->executors (reverse-map new-assignment)]] (doseq [[node+port executors] node+port->executors :let [^WorkerSlot slot (WorkerSlot. (first node+port) (last node+port)) executors (for [[start-task end-task] executors] (ExecutorDetails. start-task end-task))]] (.assign cluster slot topology-id executors))))) (defn -prepare [this conf] ) (defn -schedule [this ^Topologies topologies ^Cluster cluster] (schedule-topologies-evenly topologies cluster)) ================================================ FILE: storm-core/src/clj/backtype/storm/scheduler/IsolationScheduler.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.scheduler.IsolationScheduler (:use [backtype.storm util config log]) (:require [backtype.storm.scheduler.DefaultScheduler :as DefaultScheduler]) (:import [java.util HashSet Set List LinkedList ArrayList Map HashMap]) (:import [backtype.storm.scheduler IScheduler Topologies Cluster TopologyDetails WorkerSlot SchedulerAssignment EvenScheduler ExecutorDetails]) (:gen-class :init init :constructors {[] []} :state state :implements [backtype.storm.scheduler.IScheduler])) (defn -init [] [[] (container)]) (defn -prepare [this conf] (container-set! (.state this) conf)) (defn- compute-worker-specs "Returns mutable set of sets of executors" [^TopologyDetails details] (->> (.getExecutorToComponent details) reverse-map (map second) (apply concat) (map vector (repeat-seq (range (.getNumWorkers details)))) (group-by first) (map-val #(map second %)) vals (map set) (HashSet.) )) (defn isolated-topologies [conf topologies] (let [tset (-> conf (get ISOLATION-SCHEDULER-MACHINES) keys set)] (filter (fn [^TopologyDetails t] (contains? tset (.getName t))) topologies) )) ;; map from topology id -> set of sets of executors (defn topology-worker-specs [iso-topologies] (->> iso-topologies (map (fn [t] {(.getId t) (compute-worker-specs t)})) (apply merge))) (defn machine-distribution [conf ^TopologyDetails topology] (let [name->machines (get conf ISOLATION-SCHEDULER-MACHINES) machines (get name->machines (.getName topology)) workers (.getNumWorkers topology)] (-> (integer-divided workers machines) (dissoc 0) (HashMap.) ))) (defn topology-machine-distribution [conf iso-topologies] (->> iso-topologies (map (fn [t] {(.getId t) (machine-distribution conf t)})) (apply merge))) (defn host-assignments [^Cluster cluster] (letfn [(to-slot-specs [^SchedulerAssignment ass] (->> ass .getExecutorToSlot reverse-map (map (fn [[slot executors]] [slot (.getTopologyId ass) (set executors)]))))] (->> cluster .getAssignments vals (mapcat to-slot-specs) (group-by (fn [[^WorkerSlot slot & _]] (.getHost cluster (.getNodeId slot)))) ))) (defn- decrement-distribution! [^Map distribution value] (let [v (-> distribution (get value) dec)] (if (zero? v) (.remove distribution value) (.put distribution value v)))) ;; returns list of list of slots, reverse sorted by number of slots (defn- host-assignable-slots [^Cluster cluster] (-<> cluster .getAssignableSlots (group-by #(.getHost cluster (.getNodeId ^WorkerSlot %)) <>) (dissoc <> nil) (sort-by #(-> % second count -) <>) shuffle (LinkedList. <>) )) (defn- host->used-slots [^Cluster cluster] (->> cluster .getUsedSlots (group-by #(.getHost cluster (.getNodeId ^WorkerSlot %))) )) (defn- distribution->sorted-amts [distribution] (->> distribution (mapcat (fn [[val amt]] (repeat amt val))) (sort-by -) )) (defn- allocated-topologies [topology-worker-specs] (->> topology-worker-specs (filter (fn [[_ worker-specs]] (empty? worker-specs))) (map first) set )) (defn- leftover-topologies [^Topologies topologies filter-ids-set] (->> topologies .getTopologies (filter (fn [^TopologyDetails t] (not (contains? filter-ids-set (.getId t))))) (map (fn [^TopologyDetails t] {(.getId t) t})) (apply merge) (Topologies.) )) ;; for each isolated topology: ;; compute even distribution of executors -> workers on the number of workers specified for the topology ;; compute distribution of workers to machines ;; determine host -> list of [slot, topology id, executors] ;; iterate through hosts and: a machine is good if: ;; 1. only running workers from one isolated topology ;; 2. all workers running on it match one of the distributions of executors for that topology ;; 3. matches one of the # of workers ;; blacklist the good hosts and remove those workers from the list of need to be assigned workers ;; otherwise unassign all other workers for isolated topologies if assigned (defn remove-elem-from-set! [^Set aset] (let [elem (-> aset .iterator .next)] (.remove aset elem) elem )) ;; get host -> all assignable worker slots for non-blacklisted machines (assigned or not assigned) ;; will then have a list of machines that need to be assigned (machine -> [topology, list of list of executors]) ;; match each spec to a machine (who has the right number of workers), free everything else on that machine and assign those slots (do one topology at a time) ;; blacklist all machines who had production slots defined ;; log isolated topologies who weren't able to get enough slots / machines ;; run default scheduler on isolated topologies that didn't have enough slots + non-isolated topologies on remaining machines ;; set blacklist to what it was initially (defn -schedule [this ^Topologies topologies ^Cluster cluster] (let [conf (container-get (.state this)) orig-blacklist (HashSet. (.getBlacklistedHosts cluster)) iso-topologies (isolated-topologies conf (.getTopologies topologies)) iso-ids-set (->> iso-topologies (map #(.getId ^TopologyDetails %)) set) topology-worker-specs (topology-worker-specs iso-topologies) topology-machine-distribution (topology-machine-distribution conf iso-topologies) host-assignments (host-assignments cluster)] (doseq [[host assignments] host-assignments] (let [top-id (-> assignments first second) distribution (get topology-machine-distribution top-id) ^Set worker-specs (get topology-worker-specs top-id) num-workers (count assignments) ] (if (and (contains? iso-ids-set top-id) (every? #(= (second %) top-id) assignments) (contains? distribution num-workers) (every? #(contains? worker-specs (nth % 2)) assignments)) (do (decrement-distribution! distribution num-workers) (doseq [[_ _ executors] assignments] (.remove worker-specs executors)) (.blacklistHost cluster host)) (doseq [[slot top-id _] assignments] (when (contains? iso-ids-set top-id) (.freeSlot cluster slot) )) ))) (let [host->used-slots (host->used-slots cluster) ^LinkedList sorted-assignable-hosts (host-assignable-slots cluster)] ;; TODO: can improve things further by ordering topologies in terms of who needs the least workers (doseq [[top-id worker-specs] topology-worker-specs :let [amts (distribution->sorted-amts (get topology-machine-distribution top-id))]] (doseq [amt amts :let [[host host-slots] (.peek sorted-assignable-hosts)]] (when (and host-slots (>= (count host-slots) amt)) (.poll sorted-assignable-hosts) (.freeSlots cluster (get host->used-slots host)) (doseq [slot (take amt host-slots) :let [executors-set (remove-elem-from-set! worker-specs)]] (.assign cluster slot top-id executors-set)) (.blacklistHost cluster host)) ))) (let [failed-iso-topologies (->> topology-worker-specs (mapcat (fn [[top-id worker-specs]] (if-not (empty? worker-specs) [top-id]) )))] (if (empty? failed-iso-topologies) ;; run default scheduler on non-isolated topologies (-<> topology-worker-specs allocated-topologies (leftover-topologies topologies <>) (DefaultScheduler/default-schedule <> cluster)) (do (log-warn "Unable to isolate topologies " (pr-str failed-iso-topologies) ". No machine had enough worker slots to run the remaining workers for these topologies. Clearing all other resources and will wait for enough resources for isolated topologies before allocating any other resources.") ;; clear workers off all hosts that are not blacklisted (doseq [[host slots] (host->used-slots cluster)] (if-not (.isBlacklistedHost cluster host) (.freeSlots cluster slots) ))) )) (.setBlacklistedHosts cluster orig-blacklist) )) ================================================ FILE: storm-core/src/clj/backtype/storm/stats.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.stats (:import [backtype.storm.generated Nimbus Nimbus$Processor Nimbus$Iface StormTopology ShellComponent NotAliveException AlreadyAliveException InvalidTopologyException GlobalStreamId ClusterSummary TopologyInfo TopologySummary ExecutorSummary ExecutorStats ExecutorSpecificStats SpoutStats BoltStats ErrorInfo SupervisorSummary]) (:use [backtype.storm util]) (:use [clojure.math.numeric-tower :only [ceil]])) ;;TODO: consider replacing this with some sort of RRD (defn curr-time-bucket [^Integer time-secs ^Integer bucket-size-secs] (* bucket-size-secs (unchecked-divide-int time-secs bucket-size-secs)) ) (defrecord RollingWindow [updater merger extractor bucket-size-secs num-buckets buckets]) (defn rolling-window [updater merger extractor bucket-size-secs num-buckets] (RollingWindow. updater merger extractor bucket-size-secs num-buckets {})) (defn update-rolling-window ([^RollingWindow rw time-secs & args] ;; this is 2.5x faster than using update-in... (let [time-bucket (curr-time-bucket time-secs (:bucket-size-secs rw)) buckets (:buckets rw) curr (get buckets time-bucket) curr (apply (:updater rw) curr args) ] (assoc rw :buckets (assoc buckets time-bucket curr)) ))) (defn value-rolling-window [^RollingWindow rw] ((:extractor rw) (let [values (vals (:buckets rw))] (apply (:merger rw) values) ))) (defn cleanup-rolling-window [^RollingWindow rw] (let [buckets (:buckets rw) cutoff (- (current-time-secs) (* (:num-buckets rw) (:bucket-size-secs rw))) to-remove (filter #(< % cutoff) (keys buckets)) buckets (apply dissoc buckets to-remove)] (assoc rw :buckets buckets) )) (defn rolling-window-size [^RollingWindow rw] (* (:bucket-size-secs rw) (:num-buckets rw))) (defrecord RollingWindowSet [updater extractor windows all-time]) (defn rolling-window-set [updater merger extractor num-buckets & bucket-sizes] (RollingWindowSet. updater extractor (dofor [s bucket-sizes] (rolling-window updater merger extractor s num-buckets)) nil) ) (defn update-rolling-window-set ([^RollingWindowSet rws & args] (let [now (current-time-secs) new-windows (dofor [w (:windows rws)] (apply update-rolling-window w now args))] (assoc rws :windows new-windows :all-time (apply (:updater rws) (:all-time rws) args)) ))) (defn cleanup-rolling-window-set ([^RollingWindowSet rws] (let [windows (:windows rws)] (assoc rws :windows (map cleanup-rolling-window windows)) ))) (defn value-rolling-window-set [^RollingWindowSet rws] (merge (into {} (for [w (:windows rws)] {(rolling-window-size w) (value-rolling-window w)} )) {:all-time ((:extractor rws) (:all-time rws))})) (defn- incr-val ([amap key] (incr-val amap key 1)) ([amap key amt] (let [val (get amap key (long 0))] (assoc amap key (+ val amt)) ))) (defn- update-avg [curr val] (if curr [(+ (first curr) val) (inc (second curr))] [val (long 1)] )) (defn- merge-avg [& avg] [(apply + (map first avg)) (apply + (map second avg)) ]) (defn- extract-avg [pair] (double (/ (first pair) (second pair)))) (defn- update-keyed-avg [amap key val] (assoc amap key (update-avg (get amap key) val))) (defn- merge-keyed-avg [& vals] (apply merge-with merge-avg vals)) (defn- extract-keyed-avg [vals] (map-val extract-avg vals)) (defn- counter-extract [v] (if v v {})) (defn keyed-counter-rolling-window-set [num-buckets & bucket-sizes] (apply rolling-window-set incr-val (partial merge-with +) counter-extract num-buckets bucket-sizes)) (defn avg-rolling-window-set [num-buckets & bucket-sizes] (apply rolling-window-set update-avg merge-avg extract-avg num-buckets bucket-sizes) ) (defn keyed-avg-rolling-window-set [num-buckets & bucket-sizes] (apply rolling-window-set update-keyed-avg merge-keyed-avg extract-keyed-avg num-buckets bucket-sizes)) ;; (defn choose-bucket [val buckets] ;; (let [ret (find-first #(<= val %) buckets)] ;; (if ret ;; ret ;; (* 10 (first buckets))) ;; )) ;; ;; buckets must be between 1 and 9 ;; (defn to-proportional-bucket ;; "Maps to a bucket in the values order of magnitude. So if buckets are [1 2 5], ;; 3 -> 5 ;; 7 -> 10 ;; 1234 -> 2000 ;; etc." ;; [val buckets] ;; (cond (= 0 val) 0 ;; (between? val 1 9) (choose-bucket val buckets) ;; :else (* 10 (to-proportional-bucket (ceil (/ val 10)) ;; buckets)))) (def COMMON-FIELDS [:emitted :transferred]) (defrecord CommonStats [emitted transferred rate]) (def BOLT-FIELDS [:acked :failed :process-latencies :executed :execute-latencies]) ;;acked and failed count individual tuples (defrecord BoltExecutorStats [common acked failed process-latencies executed execute-latencies]) (def SPOUT-FIELDS [:acked :failed :complete-latencies]) ;;acked and failed count tuple completion (defrecord SpoutExecutorStats [common acked failed complete-latencies]) (def NUM-STAT-BUCKETS 20) ;; 10 minutes, 3 hours, 1 day (def STAT-BUCKETS [30 540 4320]) (defn- mk-common-stats [rate] (CommonStats. (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) rate )) (defn mk-bolt-stats [rate] (BoltExecutorStats. (mk-common-stats rate) (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) )) (defn mk-spout-stats [rate] (SpoutExecutorStats. (mk-common-stats rate) (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS)) )) (defmacro update-executor-stat! [stats path & args] (let [path (collectify path)] `(swap! (-> ~stats ~@path) update-rolling-window-set ~@args) )) (defmacro stats-rate [stats] `(-> ~stats :common :rate)) (defn emitted-tuple! [stats stream] (update-executor-stat! stats [:common :emitted] stream (stats-rate stats))) (defn transferred-tuples! [stats stream amt] (update-executor-stat! stats [:common :transferred] stream (* (stats-rate stats) amt))) (defn bolt-execute-tuple! [^BoltExecutorStats stats component stream latency-ms] (let [key [component stream]] (update-executor-stat! stats :executed key (stats-rate stats)) (update-executor-stat! stats :execute-latencies key latency-ms) )) (defn bolt-acked-tuple! [^BoltExecutorStats stats component stream latency-ms] (let [key [component stream]] (update-executor-stat! stats :acked key (stats-rate stats)) (update-executor-stat! stats :process-latencies key latency-ms) )) (defn bolt-failed-tuple! [^BoltExecutorStats stats component stream latency-ms] (let [key [component stream]] (update-executor-stat! stats :failed key (stats-rate stats)) )) (defn spout-acked-tuple! [^SpoutExecutorStats stats stream latency-ms] (update-executor-stat! stats :acked stream (stats-rate stats)) (update-executor-stat! stats :complete-latencies stream latency-ms) ) (defn spout-failed-tuple! [^SpoutExecutorStats stats stream latency-ms] (update-executor-stat! stats :failed stream (stats-rate stats)) ) (defn- cleanup-stat! [stat] (swap! stat cleanup-rolling-window-set)) (defn- cleanup-common-stats! [^CommonStats stats] (doseq [f COMMON-FIELDS] (cleanup-stat! (f stats)) )) (defn cleanup-bolt-stats! [^BoltExecutorStats stats] (cleanup-common-stats! (:common stats)) (doseq [f BOLT-FIELDS] (cleanup-stat! (f stats)) )) (defn cleanup-spout-stats! [^SpoutExecutorStats stats] (cleanup-common-stats! (:common stats)) (doseq [f SPOUT-FIELDS] (cleanup-stat! (f stats)) )) (defn- value-stats [stats fields] (into {} (dofor [f fields] [f (value-rolling-window-set @(f stats))] ))) (defn- value-common-stats [^CommonStats stats] (merge (value-stats stats COMMON-FIELDS) {:rate (:rate stats)})) (defn value-bolt-stats! [^BoltExecutorStats stats] (cleanup-bolt-stats! stats) (merge (value-common-stats (:common stats)) (value-stats stats BOLT-FIELDS) {:type :bolt})) (defn value-spout-stats! [^SpoutExecutorStats stats] (cleanup-spout-stats! stats) (merge (value-common-stats (:common stats)) (value-stats stats SPOUT-FIELDS) {:type :spout})) (defmulti render-stats! class-selector) (defmethod render-stats! SpoutExecutorStats [stats] (value-spout-stats! stats)) (defmethod render-stats! BoltExecutorStats [stats] (value-bolt-stats! stats)) (defmulti thriftify-specific-stats :type) (defn window-set-converter ([stats key-fn] ;; make the first key a string, (into {} (for [[k v] stats] [(str k) (into {} (for [[k2 v2] v] [(key-fn k2) v2]))] ) )) ([stats] (window-set-converter stats identity))) (defn to-global-stream-id [[component stream]] (GlobalStreamId. component stream) ) (defmethod thriftify-specific-stats :bolt [stats] (ExecutorSpecificStats/bolt (BoltStats. (window-set-converter (:acked stats) to-global-stream-id) (window-set-converter (:failed stats) to-global-stream-id) (window-set-converter (:process-latencies stats) to-global-stream-id) (window-set-converter (:executed stats) to-global-stream-id) (window-set-converter (:execute-latencies stats) to-global-stream-id) ))) (defmethod thriftify-specific-stats :spout [stats] (ExecutorSpecificStats/spout (SpoutStats. (window-set-converter (:acked stats)) (window-set-converter (:failed stats)) (window-set-converter (:complete-latencies stats))) )) (defn thriftify-executor-stats [stats] (let [specific-stats (thriftify-specific-stats stats)] (ExecutorStats. (window-set-converter (:emitted stats)) (window-set-converter (:transferred stats)) specific-stats))) ================================================ FILE: storm-core/src/clj/backtype/storm/testing.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.testing (:require [backtype.storm.daemon [nimbus :as nimbus] [supervisor :as supervisor] [common :as common] [worker :as worker] [executor :as executor]]) (:require [backtype.storm [process-simulator :as psim]]) (:import [org.apache.commons.io FileUtils]) (:import [java.io File]) (:import [java.util HashMap ArrayList]) (:import [java.util.concurrent.atomic AtomicInteger]) (:import [java.util.concurrent ConcurrentHashMap]) (:import [backtype.storm.utils Time Utils RegisteredGlobalState]) (:import [backtype.storm.tuple Fields Tuple TupleImpl]) (:import [backtype.storm.task TopologyContext]) (:import [backtype.storm.generated GlobalStreamId Bolt KillOptions]) (:import [backtype.storm.testing FeederSpout FixedTupleSpout FixedTuple TupleCaptureBolt SpoutTracker BoltTracker NonRichBoltTracker TestWordSpout MemoryTransactionalSpout]) (:import [backtype.storm.transactional TransactionalSpoutCoordinator]) (:import [backtype.storm.transactional.partitioned PartitionedTransactionalSpoutExecutor]) (:import [backtype.storm.tuple Tuple]) (:import [backtype.storm.generated StormTopology]) (:import [backtype.storm.task TopologyContext]) (:require [backtype.storm [zookeeper :as zk]]) (:require [backtype.storm.messaging.loader :as msg-loader]) (:require [backtype.storm.daemon.acker :as acker]) (:use [backtype.storm cluster util thrift config log])) (defn feeder-spout [fields] (FeederSpout. (Fields. fields))) (defn local-temp-path [] (str (System/getProperty "java.io.tmpdir") "/" (uuid))) (defn delete-all [paths] (dorun (for [t paths] (if (.exists (File. t)) (FileUtils/forceDelete (File. t)) )))) (defmacro with-local-tmp [[& tmp-syms] & body] (let [tmp-paths (mapcat (fn [t] [t `(local-temp-path)]) tmp-syms)] `(let [~@tmp-paths] (try ~@body (finally (delete-all ~(vec tmp-syms))) )) )) (defn start-simulating-time! [] (Time/startSimulating)) (defn stop-simulating-time! [] (Time/stopSimulating)) (defmacro with-simulated-time [& body] `(do (start-simulating-time!) (let [ret# (do ~@body)] (stop-simulating-time!) ret# ))) (defn advance-time-ms! [ms] (Time/advanceTime ms)) (defn advance-time-secs! [secs] (advance-time-ms! (* (long secs) 1000))) (defnk add-supervisor [cluster-map :ports 2 :conf {} :id nil] (let [tmp-dir (local-temp-path) port-ids (if (sequential? ports) ports (doall (repeatedly ports (:port-counter cluster-map)))) supervisor-conf (merge (:daemon-conf cluster-map) conf {STORM-LOCAL-DIR tmp-dir SUPERVISOR-SLOTS-PORTS port-ids }) id-fn (if id (fn [] id) supervisor/generate-supervisor-id) daemon (with-var-roots [supervisor/generate-supervisor-id id-fn] (supervisor/mk-supervisor supervisor-conf (:shared-context cluster-map) (supervisor/standalone-supervisor)))] (swap! (:supervisors cluster-map) conj daemon) (swap! (:tmp-dirs cluster-map) conj tmp-dir) daemon )) (defn mk-shared-context [conf] (if-not (conf STORM-LOCAL-MODE-ZMQ) (msg-loader/mk-local-context) )) ;; returns map containing cluster info ;; local dir is always overridden in maps ;; can customize the supervisors (except for ports) by passing in map for :supervisors parameter ;; if need to customize amt of ports more, can use add-supervisor calls afterwards (defnk mk-local-storm-cluster [:supervisors 2 :ports-per-supervisor 3 :daemon-conf {} :inimbus nil :supervisor-slot-port-min 1] (let [zk-tmp (local-temp-path) [zk-port zk-handle] (zk/mk-inprocess-zookeeper zk-tmp) daemon-conf (merge (read-storm-config) {TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS true ZMQ-LINGER-MILLIS 0 TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS false TOPOLOGY-TRIDENT-BATCH-EMIT-INTERVAL-MILLIS 50 } daemon-conf {STORM-CLUSTER-MODE "local" STORM-ZOOKEEPER-PORT zk-port STORM-ZOOKEEPER-SERVERS ["localhost"]}) nimbus-tmp (local-temp-path) port-counter (mk-counter supervisor-slot-port-min) nimbus (nimbus/service-handler (assoc daemon-conf STORM-LOCAL-DIR nimbus-tmp) (if inimbus inimbus (nimbus/standalone-nimbus))) context (mk-shared-context daemon-conf) cluster-map {:nimbus nimbus :port-counter port-counter :daemon-conf daemon-conf :supervisors (atom []) :state (mk-distributed-cluster-state daemon-conf) :storm-cluster-state (mk-storm-cluster-state daemon-conf) :tmp-dirs (atom [nimbus-tmp zk-tmp]) :zookeeper zk-handle :shared-context context} supervisor-confs (if (sequential? supervisors) supervisors (repeat supervisors {}))] (doseq [sc supervisor-confs] (add-supervisor cluster-map :ports ports-per-supervisor :conf sc)) cluster-map )) (defn get-supervisor [cluster-map supervisor-id] (let [finder-fn #(= (.get-id %) supervisor-id)] (find-first finder-fn @(:supervisors cluster-map)) )) (defn kill-supervisor [cluster-map supervisor-id] (let [finder-fn #(= (.get-id %) supervisor-id) supervisors @(:supervisors cluster-map) sup (find-first finder-fn supervisors)] ;; tmp-dir will be taken care of by shutdown (reset! (:supervisors cluster-map) (remove-first finder-fn supervisors)) (.shutdown sup) )) (defn kill-local-storm-cluster [cluster-map] (.shutdown (:nimbus cluster-map)) (.close (:state cluster-map)) (.disconnect (:storm-cluster-state cluster-map)) (doseq [s @(:supervisors cluster-map)] (.shutdown-all-workers s) ;; race condition here? will it launch the workers again? (supervisor/kill-supervisor s)) (psim/kill-all-processes) (log-message "Shutting down in process zookeeper") (zk/shutdown-inprocess-zookeeper (:zookeeper cluster-map)) (log-message "Done shutting down in process zookeeper") (doseq [t @(:tmp-dirs cluster-map)] (log-message "Deleting temporary path " t) (rmr t) )) (def TEST-TIMEOUT-MS 5000) (defmacro while-timeout [timeout-ms condition & body] `(let [end-time# (+ (System/currentTimeMillis) ~timeout-ms)] (while ~condition (when (> (System/currentTimeMillis) end-time#) (throw (AssertionError. (str "Test timed out (" ~timeout-ms "ms)")))) ~@body))) (defn wait-until-cluster-waiting "Wait until the cluster is idle. Should be used with time simulation." [cluster-map] ;; wait until all workers, supervisors, and nimbus is waiting (let [supervisors @(:supervisors cluster-map) workers (filter (partial satisfies? common/DaemonCommon) (psim/all-processes)) daemons (concat [(:nimbus cluster-map)] supervisors workers) ; because a worker may already be dead ] (while-timeout TEST-TIMEOUT-MS (not (every? (memfn waiting?) daemons)) (Thread/sleep 10) ;; (doseq [d daemons] ;; (if-not ((memfn waiting?) d) ;; (println d))) ))) (defn advance-cluster-time ([cluster-map secs increment-secs] (loop [left secs] (when (> left 0) (let [diff (min left increment-secs)] (advance-time-secs! diff) (wait-until-cluster-waiting cluster-map) (recur (- left diff)) )))) ([cluster-map secs] (advance-cluster-time cluster-map secs 1) )) (defmacro with-local-cluster [[cluster-sym & args] & body] `(let [~cluster-sym (mk-local-storm-cluster ~@args)] (try ~@body (catch Throwable t# (log-error t# "Error in cluster") (throw t#) ) (finally (kill-local-storm-cluster ~cluster-sym))) )) (defmacro with-simulated-time-local-cluster [& args] `(with-simulated-time (with-local-cluster ~@args))) (defmacro with-inprocess-zookeeper [port-sym & body] `(with-local-tmp [tmp#] (let [[~port-sym zks#] (zk/mk-inprocess-zookeeper tmp#)] (try ~@body (finally (zk/shutdown-inprocess-zookeeper zks#) )) ))) (defn submit-local-topology [nimbus storm-name conf topology] (when-not (Utils/isValidConf conf) (throw (IllegalArgumentException. "Topology conf is not json-serializable"))) (.submitTopology nimbus storm-name nil (to-json conf) topology)) (defn submit-local-topology-with-opts [nimbus storm-name conf topology submit-opts] (when-not (Utils/isValidConf conf) (throw (IllegalArgumentException. "Topology conf is not json-serializable"))) (.submitTopologyWithOpts nimbus storm-name nil (to-json conf) topology submit-opts)) (defn mocked-compute-new-topology->executor->node+port [storm-name executor->node+port] (fn [nimbus existing-assignments topologies scratch-topology-id] (let [topology (.getByName topologies storm-name) topology-id (.getId topology) existing-assignments (into {} (for [[tid assignment] existing-assignments] {tid (:executor->node+port assignment)})) new-assignments (assoc existing-assignments topology-id executor->node+port)] new-assignments))) (defn submit-mocked-assignment [nimbus storm-name conf topology task->component executor->node+port] (with-var-roots [common/storm-task-info (fn [& ignored] task->component) nimbus/compute-new-topology->executor->node+port (mocked-compute-new-topology->executor->node+port storm-name executor->node+port)] (submit-local-topology nimbus storm-name conf topology) )) (defn mk-capture-launch-fn [capture-atom] (fn [supervisor storm-id port worker-id] (let [supervisor-id (:supervisor-id supervisor) existing (get @capture-atom [supervisor-id port] [])] (swap! capture-atom assoc [supervisor-id port] (conj existing storm-id)) ))) (defn find-worker-id [supervisor-conf port] (let [supervisor-state (supervisor-state supervisor-conf) worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)] (first ((reverse-map worker->port) port)) )) (defn find-worker-port [supervisor-conf worker-id] (let [supervisor-state (supervisor-state supervisor-conf) worker->port (.get supervisor-state common/LS-APPROVED-WORKERS) ] (worker->port worker-id) )) (defn mk-capture-shutdown-fn [capture-atom] (let [existing-fn supervisor/shutdown-worker] (fn [supervisor worker-id] (let [conf (:conf supervisor) supervisor-id (:supervisor-id supervisor) port (find-worker-port conf worker-id) existing (get @capture-atom [supervisor-id port] 0)] (swap! capture-atom assoc [supervisor-id port] (inc existing)) (existing-fn supervisor worker-id) )))) (defmacro capture-changed-workers [& body] `(let [launch-captured# (atom {}) shutdown-captured# (atom {})] (with-var-roots [supervisor/launch-worker (mk-capture-launch-fn launch-captured#) supervisor/shutdown-worker (mk-capture-shutdown-fn shutdown-captured#)] ~@body {:launched @launch-captured# :shutdown @shutdown-captured#} ))) (defmacro capture-launched-workers [& body] `(:launched (capture-changed-workers ~@body))) (defmacro capture-shutdown-workers [& body] `(:shutdown (capture-changed-workers ~@body))) (defnk aggregated-stat [cluster-map storm-name stat-key :component-ids nil] (let [state (:storm-cluster-state cluster-map) nimbus (:nimbus cluster-map) storm-id (common/get-storm-id state storm-name) component->tasks (reverse-map (common/storm-task-info (.getUserTopology nimbus storm-id) (from-json (.getTopologyConf nimbus storm-id)))) component->tasks (if component-ids (select-keys component->tasks component-ids) component->tasks) task-ids (apply concat (vals component->tasks)) assignment (.assignment-info state storm-id nil) taskbeats (.taskbeats state storm-id (:task->node+port assignment)) heartbeats (dofor [id task-ids] (get taskbeats id)) stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))] (reduce + stats) )) (defn emitted-spout-tuples [cluster-map topology storm-name] (aggregated-stat cluster-map storm-name :emitted :component-ids (keys (.get_spouts topology)))) (defn transferred-tuples [cluster-map storm-name] (aggregated-stat cluster-map storm-name :transferred)) (defn acked-tuples [cluster-map storm-name] (aggregated-stat cluster-map storm-name :acked)) (defn simulate-wait [cluster-map] (if (Time/isSimulating) (advance-cluster-time cluster-map 10) (Thread/sleep 100) )) (defprotocol CompletableSpout (exhausted? [this] "Whether all the tuples for this spout have been completed.") (cleanup [this] "Cleanup any global state kept") (startup [this] "Prepare the spout (globally) before starting the topology")) (extend-type FixedTupleSpout CompletableSpout (exhausted? [this] (= (-> this .getSourceTuples count) (.getCompleted this))) (cleanup [this] (.cleanup this)) (startup [this] )) (extend-type TransactionalSpoutCoordinator CompletableSpout (exhausted? [this] (exhausted? (.getSpout this))) (cleanup [this] (cleanup (.getSpout this))) (startup [this] (startup (.getSpout this)))) (extend-type PartitionedTransactionalSpoutExecutor CompletableSpout (exhausted? [this] (exhausted? (.getPartitionedSpout this))) (cleanup [this] (cleanup (.getPartitionedSpout this))) (startup [this] (startup (.getPartitionedSpout this)) )) (extend-type MemoryTransactionalSpout CompletableSpout (exhausted? [this] (.isExhaustedTuples this)) (cleanup [this] (.cleanup this)) (startup [this] (.startup this))) (defn spout-objects [spec-map] (for [[_ spout-spec] spec-map] (-> spout-spec .get_spout_object deserialized-component-object))) (defn capture-topology [topology] (let [topology (.deepCopy topology) spouts (.get_spouts topology) bolts (.get_bolts topology) all-streams (apply concat (for [[id spec] (merge (clojurify-structure spouts) (clojurify-structure bolts))] (for [[stream info] (.. spec get_common get_streams)] [(GlobalStreamId. id stream) (.is_direct info)]))) capturer (TupleCaptureBolt.)] (.set_bolts topology (assoc (clojurify-structure bolts) (uuid) (Bolt. (serialize-component-object capturer) (mk-plain-component-common (into {} (for [[id direct?] all-streams] [id (if direct? (mk-direct-grouping) (mk-global-grouping))])) {} nil)) )) {:topology topology :capturer capturer} )) ;; TODO: mock-sources needs to be able to mock out state spouts as well (defnk complete-topology [cluster-map topology :mock-sources {} :storm-conf {} :cleanup-state true :topology-name nil] ;; TODO: the idea of mocking for transactional topologies should be done an ;; abstraction level above... should have a complete-transactional-topology for this (let [{topology :topology capturer :capturer} (capture-topology topology) storm-name (or topology-name (str "topologytest-" (uuid))) state (:storm-cluster-state cluster-map) spouts (.get_spouts topology) replacements (map-val (fn [v] (FixedTupleSpout. (for [tup v] (if (map? tup) (FixedTuple. (:stream tup) (:values tup)) tup)))) mock-sources) ] (doseq [[id spout] replacements] (let [spout-spec (get spouts id)] (.set_spout_object spout-spec (serialize-component-object spout)) )) (doseq [spout (spout-objects spouts)] (when-not (extends? CompletableSpout (.getClass spout)) (throw (RuntimeException. "Cannot complete topology unless every spout is a CompletableSpout (or mocked to be)")) )) (doseq [spout (spout-objects spouts)] (startup spout)) (submit-local-topology (:nimbus cluster-map) storm-name storm-conf topology) (let [storm-id (common/get-storm-id state storm-name)] (while-timeout TEST-TIMEOUT-MS (not (every? exhausted? (spout-objects spouts))) (simulate-wait cluster-map)) (.killTopologyWithOpts (:nimbus cluster-map) storm-name (doto (KillOptions.) (.set_wait_secs 0))) (while-timeout TEST-TIMEOUT-MS (.assignment-info state storm-id nil) (simulate-wait cluster-map)) (when cleanup-state (doseq [spout (spout-objects spouts)] (cleanup spout)))) (if cleanup-state (.getAndRemoveResults capturer) (.getAndClearResults capturer)) )) (defn read-tuples ([results component-id stream-id] (let [fixed-tuples (get results component-id [])] (mapcat (fn [ft] (if (= stream-id (. ft stream)) [(vec (. ft values))])) fixed-tuples) )) ([results component-id] (read-tuples results component-id Utils/DEFAULT_STREAM_ID) )) (defn ms= [& args] (apply = (map multi-set args))) (def TRACKER-BOLT-ID "+++tracker-bolt") ;; TODO: should override system-topology! and wrap everything there (defn mk-tracked-topology ([tracked-cluster topology] (let [track-id (::track-id tracked-cluster) ret (.deepCopy topology)] (dofor [[_ bolt] (.get_bolts ret) :let [obj (deserialized-component-object (.get_bolt_object bolt))]] (.set_bolt_object bolt (serialize-component-object (BoltTracker. obj track-id)))) (dofor [[_ spout] (.get_spouts ret) :let [obj (deserialized-component-object (.get_spout_object spout))]] (.set_spout_object spout (serialize-component-object (SpoutTracker. obj track-id)))) {:topology ret :last-spout-emit (atom 0) :cluster tracked-cluster } ))) (defn assoc-track-id [cluster track-id] (assoc cluster ::track-id track-id)) (defn increment-global! [id key amt] (-> (RegisteredGlobalState/getState id) (get key) (.addAndGet amt))) (defn global-amt [id key] (-> (RegisteredGlobalState/getState id) (get key) .get )) (defmacro with-tracked-cluster [[cluster-sym & cluster-args] & body] `(let [id# (uuid)] (RegisteredGlobalState/setState id# (doto (ConcurrentHashMap.) (.put "spout-emitted" (AtomicInteger. 0)) (.put "transferred" (AtomicInteger. 0)) (.put "processed" (AtomicInteger. 0)))) (with-var-roots [acker/mk-acker-bolt (let [old# acker/mk-acker-bolt] (fn [& args#] (NonRichBoltTracker. (apply old# args#) id#) )) ;; critical that this particular function is overridden here, ;; since the transferred stat needs to be incremented at the moment ;; of tuple emission (and not on a separate thread later) for ;; topologies to be tracked correctly. This is because "transferred" *must* ;; be incremented before "processing". executor/mk-executor-transfer-fn (let [old# executor/mk-executor-transfer-fn] (fn [& args#] (let [transferrer# (apply old# args#)] (fn [& args2#] ;; (log-message "Transferring: " transfer-args#) (increment-global! id# "transferred" 1) (apply transferrer# args2#) )))) ] (with-local-cluster [~cluster-sym ~@cluster-args] (let [~cluster-sym (assoc-track-id ~cluster-sym id#)] ~@body) )) (RegisteredGlobalState/clearState id#) )) (defn tracked-wait "Waits until topology is idle and 'amt' more tuples have been emitted by spouts." ([tracked-topology] (tracked-wait tracked-topology 1)) ([tracked-topology amt] (let [target (+ amt @(:last-spout-emit tracked-topology)) track-id (-> tracked-topology :cluster ::track-id) waiting? (fn [] (or (not= target (global-amt track-id "spout-emitted")) (not= (global-amt track-id "transferred") (global-amt track-id "processed")) ))] (while-timeout TEST-TIMEOUT-MS (waiting?) ;; (println "Spout emitted: " (global-amt track-id "spout-emitted")) ;; (println "Processed: " (global-amt track-id "processed")) ;; (println "Transferred: " (global-amt track-id "transferred")) (Thread/sleep 500)) (reset! (:last-spout-emit tracked-topology) target) ))) (defnk test-tuple [values :stream Utils/DEFAULT_STREAM_ID :component "component" :fields nil] (let [fields (or fields (->> (iterate inc 1) (take (count values)) (map #(str "field" %)))) spout-spec (mk-spout-spec* (TestWordSpout.) {stream fields}) topology (StormTopology. {component spout-spec} {} {}) context (TopologyContext. topology (read-storm-config) {(int 1) component} {component [(int 1)]} {component {stream (Fields. fields)}} "test-storm-id" nil nil (int 1) nil [(int 1)] {} {} (HashMap.) (HashMap.) (atom false))] (TupleImpl. context values 1 stream) )) ================================================ FILE: storm-core/src/clj/backtype/storm/testing4j.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.testing4j (:import [java.util Map List Collection ArrayList]) (:import [backtype.storm Config ILocalCluster LocalCluster]) (:import [backtype.storm.generated StormTopology]) (:import [backtype.storm.daemon nimbus]) (:import [backtype.storm.testing TestJob MockedSources TrackedTopology MkClusterParam CompleteTopologyParam MkTupleParam]) (:import [backtype.storm.utils Utils]) (:use [backtype.storm testing util log]) (:gen-class :name backtype.storm.Testing :methods [^:static [completeTopology [backtype.storm.ILocalCluster backtype.storm.generated.StormTopology backtype.storm.testing.CompleteTopologyParam] java.util.Map] ^:static [completeTopology [backtype.storm.ILocalCluster backtype.storm.generated.StormTopology] java.util.Map] ^:static [withSimulatedTime [Runnable] void] ^:static [withLocalCluster [backtype.storm.testing.TestJob] void] ^:static [withLocalCluster [backtype.storm.testing.MkClusterParam backtype.storm.testing.TestJob] void] ^:static [withSimulatedTimeLocalCluster [backtype.storm.testing.TestJob] void] ^:static [withSimulatedTimeLocalCluster [backtype.storm.testing.MkClusterParam backtype.storm.testing.TestJob] void] ^:static [withTrackedCluster [backtype.storm.testing.TestJob] void] ^:static [withTrackedCluster [backtype.storm.testing.MkClusterParam backtype.storm.testing.TestJob] void] ^:static [readTuples [java.util.Map String String] java.util.List] ^:static [readTuples [java.util.Map String] java.util.List] ^:static [mkTrackedTopology [backtype.storm.ILocalCluster backtype.storm.generated.StormTopology] backtype.storm.testing.TrackedTopology] ^:static [trackedWait [backtype.storm.testing.TrackedTopology] void] ^:static [trackedWait [backtype.storm.testing.TrackedTopology Integer] void] ^:static [advanceClusterTime [backtype.storm.ILocalCluster Integer Integer] void] ^:static [advanceClusterTime [backtype.storm.ILocalCluster Integer] void] ^:static [multiseteq [java.util.Collection java.util.Collection] boolean] ^:static [multiseteq [java.util.Map java.util.Map] boolean] ^:static [testTuple [java.util.List] backtype.storm.tuple.Tuple] ^:static [testTuple [java.util.List backtype.storm.testing.MkTupleParam] backtype.storm.tuple.Tuple]])) (defn -completeTopology ([^ILocalCluster cluster ^StormTopology topology ^CompleteTopologyParam completeTopologyParam] (let [mocked-sources (or (-> completeTopologyParam .getMockedSources .getData) {}) storm-conf (or (.getStormConf completeTopologyParam) {}) cleanup-state (or (.getCleanupState completeTopologyParam) true) topology-name (.getTopologyName completeTopologyParam)] (complete-topology (.getState cluster) topology :mock-sources mocked-sources :storm-conf storm-conf :cleanup-state cleanup-state :topology-name topology-name))) ([^ILocalCluster cluster ^StormTopology topology] (-completeTopology cluster topology (CompleteTopologyParam.)))) (defn -withSimulatedTime [^Runnable code] (with-simulated-time (.run code))) (defmacro with-cluster [cluster-type mkClusterParam code] `(let [supervisors# (or (.getSupervisors ~mkClusterParam) 2) ports-per-supervisor# (or (.getPortsPerSupervisor ~mkClusterParam) 3) daemon-conf# (or (.getDaemonConf ~mkClusterParam) {})] (~cluster-type [cluster# :supervisors supervisors# :ports-per-supervisor ports-per-supervisor# :daemon-conf daemon-conf#] (let [cluster# (LocalCluster. cluster#)] (.run ~code cluster#))))) (defn -withLocalCluster ([^MkClusterParam mkClusterParam ^TestJob code] (with-cluster with-local-cluster mkClusterParam code)) ([^TestJob code] (-withLocalCluster (MkClusterParam.) code))) (defn -withSimulatedTimeLocalCluster ([^MkClusterParam mkClusterParam ^TestJob code] (with-cluster with-simulated-time-local-cluster mkClusterParam code)) ([^TestJob code] (-withSimulatedTimeLocalCluster (MkClusterParam.) code))) (defn -withTrackedCluster ([^MkClusterParam mkClusterParam ^TestJob code] (with-cluster with-tracked-cluster mkClusterParam code)) ([^TestJob code] (-withTrackedCluster (MkClusterParam.) code))) (defn- find-tuples [^List fixed-tuples ^String stream] (let [ret (ArrayList.)] (doseq [fixed-tuple fixed-tuples] (if (= (.stream fixed-tuple) stream) (.add ret (.values fixed-tuple)))) ret)) (defn -readTuples ([^Map result ^String componentId ^String streamId] (let [stream-result (.get result componentId) ret (if stream-result (find-tuples stream-result streamId) [])] ret)) ([^Map result ^String componentId] (-readTuples result componentId Utils/DEFAULT_STREAM_ID))) (defn -mkTrackedTopology [^ILocalCluster trackedCluster ^StormTopology topology] (-> (mk-tracked-topology (.getState trackedCluster) topology) (TrackedTopology.))) (defn -trackedWait ([^TrackedTopology trackedTopology ^Integer amt] (tracked-wait trackedTopology amt)) ([^TrackedTopology trackedTopology] (-trackedWait trackedTopology 1))) (defn -advanceClusterTime ([^ILocalCluster cluster ^Integer secs ^Integer step] (advance-cluster-time (.getState cluster) secs step)) ([^ILocalCluster cluster ^Integer secs] (-advanceClusterTime cluster secs 1))) (defn- multiseteq [^Object obj1 ^Object obj2] (let [obj1 (clojurify-structure obj1) obj2 (clojurify-structure obj2)] (ms= obj1 obj2))) (defn -multiseteq [^Collection coll1 ^Collection coll2] (multiseteq coll1 coll2)) (defn -multiseteq [^Map coll1 ^Map coll2] (multiseteq coll1 coll2)) (defn -testTuple ([^List values] (-testTuple values nil)) ([^List values ^MkTupleParam param] (if (nil? param) (test-tuple values) (let [stream (or (.getStream param) Utils/DEFAULT_STREAM_ID) component (or (.getComponent param) "component") fields (.getFields param)] (test-tuple values :stream stream :component component :fields fields))))) ================================================ FILE: storm-core/src/clj/backtype/storm/thrift.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.thrift (:import [java.util HashMap]) (:import [backtype.storm.generated JavaObject Grouping Nimbus StormTopology StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo GlobalStreamId ComponentObject ComponentObject$_Fields ShellComponent]) (:import [backtype.storm.utils Utils]) (:import [backtype.storm Constants]) (:import [backtype.storm.grouping CustomStreamGrouping]) (:import [backtype.storm.topology TopologyBuilder]) (:import [backtype.storm.clojure RichShellBolt RichShellSpout]) (:import [org.apache.thrift7.protocol TBinaryProtocol TProtocol]) (:import [org.apache.thrift7.transport TTransport TFramedTransport TSocket]) (:use [backtype.storm util config log]) ) (defn instantiate-java-object [^JavaObject obj] (let [name (symbol (.get_full_class_name obj)) args (map (memfn getFieldValue) (.get_args_list obj))] (eval `(new ~name ~@args)) )) (def grouping-constants {Grouping$_Fields/FIELDS :fields Grouping$_Fields/SHUFFLE :shuffle Grouping$_Fields/ALL :all Grouping$_Fields/NONE :none Grouping$_Fields/CUSTOM_SERIALIZED :custom-serialized Grouping$_Fields/CUSTOM_OBJECT :custom-object Grouping$_Fields/DIRECT :direct Grouping$_Fields/LOCAL_OR_SHUFFLE :local-or-shuffle }) (defn grouping-type [^Grouping grouping] (grouping-constants (.getSetField grouping))) (defn field-grouping [^Grouping grouping] (when-not (= (grouping-type grouping) :fields) (throw (IllegalArgumentException. "Tried to get grouping fields from non fields grouping"))) (.get_fields grouping)) (defn global-grouping? [^Grouping grouping] (and (= :fields (grouping-type grouping)) (empty? (field-grouping grouping)) )) (defn parallelism-hint [^ComponentCommon component-common] (let [phint (.get_parallelism_hint component-common)] (if-not (.is_set_parallelism_hint component-common) 1 phint) )) (defn nimbus-client-and-conn [host port] (log-message "Connecting to Nimbus at " host ":" port) (let [transport (TFramedTransport. (TSocket. host port)) prot (TBinaryProtocol. transport) client (Nimbus$Client. prot)] (.open transport) [client transport] )) (defmacro with-nimbus-connection [[client-sym host port] & body] `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)] (try ~@body (finally (.close conn#))) )) (defmacro with-configured-nimbus-connection [client-sym & body] `(let [conf# (read-storm-config) host# (conf# NIMBUS-HOST) port# (conf# NIMBUS-THRIFT-PORT)] (with-nimbus-connection [~client-sym host# port#] ~@body ))) (defn direct-output-fields [fields] (StreamInfo. fields true)) (defn output-fields [fields] (StreamInfo. fields false)) (defn mk-output-spec [output-spec] (let [output-spec (if (map? output-spec) output-spec {Utils/DEFAULT_STREAM_ID output-spec})] (map-val (fn [out] (if (instance? StreamInfo out) out (StreamInfo. out false) )) output-spec ))) (defnk mk-plain-component-common [inputs output-spec parallelism-hint :conf nil] (let [ret (ComponentCommon. (HashMap. inputs) (HashMap. (mk-output-spec output-spec)))] (when parallelism-hint (.set_parallelism_hint ret parallelism-hint)) (when conf (.set_json_conf ret (to-json conf))) ret )) (defnk mk-spout-spec* [spout outputs :p nil :conf nil] (SpoutSpec. (ComponentObject/serialized_java (Utils/serialize spout)) (mk-plain-component-common {} outputs p :conf conf))) (defn mk-shuffle-grouping [] (Grouping/shuffle (NullStruct.))) (defn mk-local-or-shuffle-grouping [] (Grouping/local_or_shuffle (NullStruct.))) (defn mk-fields-grouping [fields] (Grouping/fields fields)) (defn mk-global-grouping [] (mk-fields-grouping [])) (defn mk-direct-grouping [] (Grouping/direct (NullStruct.))) (defn mk-all-grouping [] (Grouping/all (NullStruct.))) (defn mk-none-grouping [] (Grouping/none (NullStruct.))) (defn deserialized-component-object [^ComponentObject obj] (when (not= (.getSetField obj) ComponentObject$_Fields/SERIALIZED_JAVA) (throw (RuntimeException. "Cannot deserialize non-java-serialized object"))) (Utils/deserialize (.get_serialized_java obj)) ) (defn serialize-component-object [obj] (ComponentObject/serialized_java (Utils/serialize obj))) (defn- mk-grouping [grouping-spec] (cond (nil? grouping-spec) (mk-none-grouping) (instance? Grouping grouping-spec) grouping-spec (instance? CustomStreamGrouping grouping-spec) (Grouping/custom_serialized (Utils/serialize grouping-spec)) (instance? JavaObject grouping-spec) (Grouping/custom_object grouping-spec) (sequential? grouping-spec) (mk-fields-grouping grouping-spec) (= grouping-spec :shuffle) (mk-shuffle-grouping) (= grouping-spec :local-or-shuffle) (mk-local-or-shuffle-grouping) (= grouping-spec :none) (mk-none-grouping) (= grouping-spec :all) (mk-all-grouping) (= grouping-spec :global) (mk-global-grouping) (= grouping-spec :direct) (mk-direct-grouping) true (throw (IllegalArgumentException. (str grouping-spec " is not a valid grouping"))) )) (defn- mk-inputs [inputs] (into {} (for [[stream-id grouping-spec] inputs] [(if (sequential? stream-id) (GlobalStreamId. (first stream-id) (second stream-id)) (GlobalStreamId. stream-id Utils/DEFAULT_STREAM_ID)) (mk-grouping grouping-spec)] ))) (defnk mk-bolt-spec* [inputs bolt outputs :p nil :conf nil] (let [common (mk-plain-component-common (mk-inputs inputs) outputs p :conf conf)] (Bolt. (ComponentObject/serialized_java (Utils/serialize bolt)) common ))) (defnk mk-spout-spec [spout :parallelism-hint nil :p nil :conf nil] (let [parallelism-hint (if p p parallelism-hint)] {:obj spout :p parallelism-hint :conf conf} )) (defn- shell-component-params [command script-or-output-spec kwargs] (if (string? script-or-output-spec) [(into-array String [command script-or-output-spec]) (first kwargs) (rest kwargs)] [(into-array String command) script-or-output-spec kwargs])) (defnk mk-bolt-spec [inputs bolt :parallelism-hint nil :p nil :conf nil] (let [parallelism-hint (if p p parallelism-hint)] {:obj bolt :inputs inputs :p parallelism-hint :conf conf} )) (defn mk-shell-bolt-spec [inputs command script-or-output-spec & kwargs] (let [[command output-spec kwargs] (shell-component-params command script-or-output-spec kwargs)] (apply mk-bolt-spec inputs (RichShellBolt. command (mk-output-spec output-spec)) kwargs))) (defn mk-shell-spout-spec [command script-or-output-spec & kwargs] (let [[command output-spec kwargs] (shell-component-params command script-or-output-spec kwargs)] (apply mk-spout-spec (RichShellSpout. command (mk-output-spec output-spec)) kwargs))) (defn- add-inputs [declarer inputs] (doseq [[id grouping] (mk-inputs inputs)] (.grouping declarer id grouping) )) (defn mk-topology ([spout-map bolt-map] (let [builder (TopologyBuilder.)] (doseq [[name {spout :obj p :p conf :conf}] spout-map] (-> builder (.setSpout name spout (if-not (nil? p) (int p) p)) (.addConfigurations conf))) (doseq [[name {bolt :obj p :p conf :conf inputs :inputs}] bolt-map] (-> builder (.setBolt name bolt (if-not (nil? p) (int p) p)) (.addConfigurations conf) (add-inputs inputs))) (.createTopology builder) )) ([spout-map bolt-map state-spout-map] (mk-topology spout-map bolt-map))) ;; clojurify-structure is needed or else every element becomes the same after successive calls ;; don't know why this happens (def STORM-TOPOLOGY-FIELDS (-> StormTopology/metaDataMap clojurify-structure keys)) (def SPOUT-FIELDS [StormTopology$_Fields/SPOUTS StormTopology$_Fields/STATE_SPOUTS]) ================================================ FILE: storm-core/src/clj/backtype/storm/timer.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.timer (:import [backtype.storm.utils Time]) (:import [java.util PriorityQueue Comparator]) (:import [java.util.concurrent Semaphore]) (:use [backtype.storm util log]) ) ;; The timer defined in this file is very similar to java.util.Timer, except it integrates with ;; Storm's time simulation capabilities. This lets us test code that does asynchronous work on the timer thread (defnk mk-timer [:kill-fn (fn [& _] )] (let [queue (PriorityQueue. 10 (reify Comparator (compare [this o1 o2] (- (first o1) (first o2)) ) (equals [this obj] true ))) active (atom true) lock (Object.) notifier (Semaphore. 0) timer-thread (Thread. (fn [] (while @active (try (let [[time-millis _ _ :as elem] (locking lock (.peek queue))] (if (and elem (>= (current-time-millis) time-millis)) ;; imperative to not run the function inside the timer lock ;; otherwise, it's possible to deadlock if function deals with other locks ;; (like the submit lock) (let [afn (locking lock (second (.poll queue)))] (afn)) (if time-millis ;; if any events are scheduled ;; sleep until event generation ;; note that if any recurring events are scheduled then we will always go through ;; this branch, sleeping only the exact necessary amount of time (Time/sleep (- time-millis (current-time-millis))) ;; else poll to see if any new event was scheduled ;; this is in essence the response time for detecting any new event schedulings when ;; there are no scheduled events (Time/sleep 1000)) )) (catch Throwable t ;; because the interrupted exception can be wrapped in a runtimeexception (when-not (exception-cause? InterruptedException t) (kill-fn t) (reset! active false) (throw t)) ))) (.release notifier)))] (.setDaemon timer-thread true) (.setPriority timer-thread Thread/MAX_PRIORITY) (.start timer-thread) {:timer-thread timer-thread :queue queue :active active :lock lock :cancel-notifier notifier})) (defn- check-active! [timer] (when-not @(:active timer) (throw (IllegalStateException. "Timer is not active")))) (defnk schedule [timer delay-secs afn :check-active true] (when check-active (check-active! timer)) (let [id (uuid) ^PriorityQueue queue (:queue timer)] (locking (:lock timer) (.add queue [(+ (current-time-millis) (* 1000 (long delay-secs))) afn id]) ))) (defn schedule-recurring [timer delay-secs recur-secs afn] (schedule timer delay-secs (fn this [] (afn) (schedule timer recur-secs this :check-active false)) ; this avoids a race condition with cancel-timer )) (defn cancel-timer [timer] (check-active! timer) (locking (:lock timer) (reset! (:active timer) false) (.interrupt (:timer-thread timer))) (.acquire (:cancel-notifier timer))) (defn timer-waiting? [timer] (Time/isThreadWaiting (:timer-thread timer))) ================================================ FILE: storm-core/src/clj/backtype/storm/tuple.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.tuple (:use [backtype.storm bootstrap]) ) (bootstrap) (defn list-hash-code [^List alist] (.hashCode alist)) ================================================ FILE: storm-core/src/clj/backtype/storm/ui/core.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.ui.core (:use compojure.core) (:use ring.middleware.reload) (:use [hiccup core page-helpers]) (:use [backtype.storm config util log]) (:use [backtype.storm.ui helpers]) (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID system-id?]]]) (:use [ring.adapter.jetty :only [run-jetty]]) (:use [clojure.string :only [trim]]) (:import [backtype.storm.generated ExecutorSpecificStats ExecutorStats ExecutorSummary TopologyInfo SpoutStats BoltStats ErrorInfo ClusterSummary SupervisorSummary TopologySummary Nimbus$Client StormTopology GlobalStreamId RebalanceOptions KillOptions]) (:import [java.io File]) (:require [compojure.route :as route] [compojure.handler :as handler] [ring.util.response :as resp] [backtype.storm [thrift :as thrift]]) (:import [org.apache.commons.lang StringEscapeUtils]) (:gen-class)) (def ^:dynamic *STORM-CONF* (read-storm-config)) (defmacro with-nimbus [nimbus-sym & body] `(thrift/with-nimbus-connection [~nimbus-sym (*STORM-CONF* NIMBUS-HOST) (*STORM-CONF* NIMBUS-THRIFT-PORT)] ~@body )) (defn get-filled-stats [summs] (->> summs (map #(.get_stats ^ExecutorSummary %)) (filter not-nil?))) (defn mk-system-toggle-button [include-sys?] [:p {:class "js-only"} [:input {:type "button" :value (str (if include-sys? "Hide" "Show") " System Stats") :onclick "toggleSys()"}]]) (defn ui-template [body] (html4 [:head [:title "Storm UI"] (include-css "/css/bootstrap-1.1.0.css") (include-css "/css/style.css") (include-js "/js/jquery-1.6.2.min.js") (include-js "/js/jquery.tablesorter.min.js") (include-js "/js/jquery.cookies.2.2.0.min.js") (include-js "/js/script.js") ] [:body [:h1 (link-to "/" "Storm UI")] (seq body) ])) (defn read-storm-version [] (let [storm-home (System/getProperty "storm.home") release-path (format "%s/RELEASE" storm-home) release-file (File. release-path)] (if (and (.exists release-file) (.isFile release-file)) (trim (slurp release-path)) "Unknown"))) (defn cluster-summary-table [^ClusterSummary summ] (let [sups (.get_supervisors summ) used-slots (reduce + (map #(.get_num_used_workers ^SupervisorSummary %) sups)) total-slots (reduce + (map #(.get_num_workers ^SupervisorSummary %) sups)) free-slots (- total-slots used-slots) total-tasks (->> (.get_topologies summ) (map #(.get_num_tasks ^TopologySummary %)) (reduce +)) total-executors (->> (.get_topologies summ) (map #(.get_num_executors ^TopologySummary %)) (reduce +))] (table ["Version" "Nimbus uptime" "Supervisors" "Used slots" "Free slots" "Total slots" "Executors" "Tasks"] [[(read-storm-version) (pretty-uptime-sec (.get_nimbus_uptime_secs summ)) (count sups) used-slots free-slots total-slots total-executors total-tasks]]) )) (defn topology-link ([id] (topology-link id id)) ([id content] (link-to (url-format "/topology/%s" id) (escape-html content)))) (defn main-topology-summary-table [summs] ;; make the id clickable ;; make the table sortable (sorted-table ["Name" "Id" "Status" "Uptime" "Num workers" "Num executors" "Num tasks"] (for [^TopologySummary t summs] [(topology-link (.get_id t) (.get_name t)) (escape-html (.get_id t)) (.get_status t) (pretty-uptime-sec (.get_uptime_secs t)) (.get_num_workers t) (.get_num_executors t) (.get_num_tasks t) ]) :time-cols [3] :sort-list "[[0,0]]" )) (defn supervisor-summary-table [summs] (sorted-table ["Id" "Host" "Uptime" "Slots" "Used slots"] (for [^SupervisorSummary s summs] [(.get_supervisor_id s) (.get_host s) (pretty-uptime-sec (.get_uptime_secs s)) (.get_num_workers s) (.get_num_used_workers s)]) :time-cols [2])) (defn configuration-table [conf] (sorted-table ["Key" "Value"] (map #(vector (key %) (str (val %))) conf))) (defn main-page [] (with-nimbus nimbus (let [summ (.getClusterInfo ^Nimbus$Client nimbus)] (concat [[:h2 "Cluster Summary"]] [(cluster-summary-table summ)] [[:h2 "Topology summary"]] (main-topology-summary-table (.get_topologies summ)) [[:h2 "Supervisor summary"]] (supervisor-summary-table (.get_supervisors summ)) [[:h2 "Nimbus Configuration"]] (configuration-table (from-json (.getNimbusConf ^Nimbus$Client nimbus))) )))) (defn component-type [^StormTopology topology id] (let [bolts (.get_bolts topology) spouts (.get_spouts topology)] (cond (.containsKey bolts id) :bolt (.containsKey spouts id) :spout ))) (defn executor-summary-type [topology ^ExecutorSummary s] (component-type topology (.get_component_id s))) (defn add-pairs ([] [0 0]) ([[a1 a2] [b1 b2]] [(+ a1 b1) (+ a2 b2)])) (defn expand-averages [avg counts] (let [avg (clojurify-structure avg) counts (clojurify-structure counts)] (into {} (for [[slice streams] counts] [slice (into {} (for [[stream c] streams] [stream [(* c (get-in avg [slice stream])) c]] ))] )))) (defn expand-averages-seq [average-seq counts-seq] (->> (map vector average-seq counts-seq) (map #(apply expand-averages %)) (apply merge-with (fn [s1 s2] (merge-with add-pairs s1 s2))) )) (defn- val-avg [[t c]] (if (= t 0) 0 (double (/ t c)))) (defn aggregate-averages [average-seq counts-seq] (->> (expand-averages-seq average-seq counts-seq) (map-val (fn [s] (map-val val-avg s) )) )) (defn aggregate-counts [counts-seq] (->> counts-seq (map clojurify-structure) (apply merge-with (fn [s1 s2] (merge-with + s1 s2)) ))) (defn aggregate-avg-streams [avg counts] (let [expanded (expand-averages avg counts)] (->> expanded (map-val #(reduce add-pairs (vals %))) (map-val val-avg) ))) (defn aggregate-count-streams [stats] (->> stats (map-val #(reduce + (vals %))))) (defn aggregate-common-stats [stats-seq] {:emitted (aggregate-counts (map #(.get_emitted ^ExecutorStats %) stats-seq)) :transferred (aggregate-counts (map #(.get_transferred ^ExecutorStats %) stats-seq))} ) (defn mk-include-sys-fn [include-sys?] (if include-sys? (fn [_] true) (fn [stream] (and (string? stream) (not (system-id? stream)))))) (defn pre-process [stream-summary include-sys?] (let [filter-fn (mk-include-sys-fn include-sys?) emitted (:emitted stream-summary) emitted (into {} (for [[window stat] emitted] {window (filter-key filter-fn stat)})) transferred (:transferred stream-summary) transferred (into {} (for [[window stat] transferred] {window (filter-key filter-fn stat)})) stream-summary (-> stream-summary (dissoc :emitted) (assoc :emitted emitted)) stream-summary (-> stream-summary (dissoc :transferred) (assoc :transferred transferred))] stream-summary)) (defn aggregate-bolt-stats [stats-seq include-sys?] (let [stats-seq (collectify stats-seq)] (merge (pre-process (aggregate-common-stats stats-seq) include-sys?) {:acked (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_acked) stats-seq)) :failed (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_failed) stats-seq)) :executed (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_executed) stats-seq)) :process-latencies (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_process_ms_avg) stats-seq) (map #(.. ^ExecutorStats % get_specific get_bolt get_acked) stats-seq)) :execute-latencies (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_execute_ms_avg) stats-seq) (map #(.. ^ExecutorStats % get_specific get_bolt get_executed) stats-seq)) }))) (defn aggregate-spout-stats [stats-seq include-sys?] (let [stats-seq (collectify stats-seq)] (merge (pre-process (aggregate-common-stats stats-seq) include-sys?) {:acked (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_acked) stats-seq)) :failed (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_failed) stats-seq)) :complete-latencies (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_spout get_complete_ms_avg) stats-seq) (map #(.. ^ExecutorStats % get_specific get_spout get_acked) stats-seq)) } ))) (defn aggregate-bolt-streams [stats] {:acked (aggregate-count-streams (:acked stats)) :failed (aggregate-count-streams (:failed stats)) :emitted (aggregate-count-streams (:emitted stats)) :transferred (aggregate-count-streams (:transferred stats)) :process-latencies (aggregate-avg-streams (:process-latencies stats) (:acked stats)) :executed (aggregate-count-streams (:executed stats)) :execute-latencies (aggregate-avg-streams (:execute-latencies stats) (:executed stats)) }) (defn aggregate-spout-streams [stats] {:acked (aggregate-count-streams (:acked stats)) :failed (aggregate-count-streams (:failed stats)) :emitted (aggregate-count-streams (:emitted stats)) :transferred (aggregate-count-streams (:transferred stats)) :complete-latencies (aggregate-avg-streams (:complete-latencies stats) (:acked stats)) }) (defn spout-summary? [topology s] (= :spout (executor-summary-type topology s))) (defn bolt-summary? [topology s] (= :bolt (executor-summary-type topology s))) (defn topology-summary-table [^TopologyInfo summ] (let [executors (.get_executors summ) workers (set (for [^ExecutorSummary e executors] [(.get_host e) (.get_port e)]))] (table ["Name" "Id" "Status" "Uptime" "Num workers" "Num executors" "Num tasks"] [[(escape-html (.get_name summ)) (escape-html (.get_id summ)) (.get_status summ) (pretty-uptime-sec (.get_uptime_secs summ)) (count workers) (count executors) (sum-tasks executors) ]] ))) (defn total-aggregate-stats [spout-summs bolt-summs include-sys?] (let [spout-stats (get-filled-stats spout-summs) bolt-stats (get-filled-stats bolt-summs) agg-spout-stats (-> spout-stats (aggregate-spout-stats include-sys?) aggregate-spout-streams) agg-bolt-stats (-> bolt-stats (aggregate-bolt-stats include-sys?) aggregate-bolt-streams)] (merge-with (fn [s1 s2] (merge-with + s1 s2)) (select-keys agg-bolt-stats [:emitted :transferred]) agg-spout-stats ))) (defn stats-times [stats-map] (sort-by #(Integer/parseInt %) (-> stats-map clojurify-structure (dissoc ":all-time") keys))) (defn topology-stats-table [id window stats] (let [times (stats-times (:emitted stats)) display-map (into {} (for [t times] [t pretty-uptime-sec])) display-map (assoc display-map ":all-time" (fn [_] "All time"))] (sorted-table ["Window" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed"] (for [k (concat times [":all-time"]) :let [disp ((display-map k) k)]] [(link-to (if (= k window) {:class "red"} {}) (url-format "/topology/%s?window=%s" id k) disp) (get-in stats [:emitted k]) (get-in stats [:transferred k]) (float-str (get-in stats [:complete-latencies k])) (get-in stats [:acked k]) (get-in stats [:failed k]) ] ) :time-cols [0] ))) (defn group-by-comp [summs] (let [ret (group-by #(.get_component_id ^ExecutorSummary %) summs)] (into (sorted-map) ret ))) (defn error-subset [error-str] (apply str (take 200 error-str))) (defn most-recent-error [errors-list] (let [error (->> errors-list (sort-by #(.get_error_time_secs ^ErrorInfo %)) reverse first)] (if error [:span (if (< (time-delta (.get_error_time_secs ^ErrorInfo error)) (* 60 30)) {:class "red"} {}) (error-subset (.get_error ^ErrorInfo error))] ))) (defn component-link [storm-id id] (link-to (url-format "/topology/%s/component/%s" storm-id id) (escape-html id))) (defn worker-log-link [host port] (link-to (url-format "http://%s:%s/log?file=worker-%s.log" host (*STORM-CONF* LOGVIEWER-PORT) port) (str port))) (defn render-capacity [capacity] (let [capacity (nil-to-zero capacity)] [:span (if (> capacity 0.9) {:class "red"} {}) (float-str capacity)])) (defn compute-executor-capacity [^ExecutorSummary e] (let [stats (.get_stats e) stats (if stats (-> stats (aggregate-bolt-stats true) (aggregate-bolt-streams) swap-map-order (get "600"))) uptime (nil-to-zero (.get_uptime_secs e)) window (if (< uptime 600) uptime 600) executed (-> stats :executed nil-to-zero) latency (-> stats :execute-latencies nil-to-zero) ] (if (> window 0) (div (* executed latency) (* 1000 window)) ))) (defn compute-bolt-capacity [executors] (->> executors (map compute-executor-capacity) (map nil-to-zero) (apply max))) (defn spout-comp-table [top-id summ-map errors window include-sys?] (sorted-table ["Id" "Executors" "Tasks" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed" "Last error"] (for [[id summs] summ-map :let [stats-seq (get-filled-stats summs) stats (aggregate-spout-streams (aggregate-spout-stats stats-seq include-sys?))]] [(component-link top-id id) (count summs) (sum-tasks summs) (get-in stats [:emitted window]) (get-in stats [:transferred window]) (float-str (get-in stats [:complete-latencies window])) (get-in stats [:acked window]) (get-in stats [:failed window]) (most-recent-error (get errors id)) ] ))) (defn bolt-comp-table [top-id summ-map errors window include-sys?] (sorted-table ["Id" "Executors" "Tasks" "Emitted" "Transferred" "Capacity (last 10m)" "Execute latency (ms)" "Executed" "Process latency (ms)" "Acked" "Failed" "Last error"] (for [[id summs] summ-map :let [stats-seq (get-filled-stats summs) stats (aggregate-bolt-streams (aggregate-bolt-stats stats-seq include-sys?)) ]] [(component-link top-id id) (count summs) (sum-tasks summs) (get-in stats [:emitted window]) (get-in stats [:transferred window]) (render-capacity (compute-bolt-capacity summs)) (float-str (get-in stats [:execute-latencies window])) (get-in stats [:executed window]) (float-str (get-in stats [:process-latencies window])) (get-in stats [:acked window]) (get-in stats [:failed window]) (most-recent-error (get errors id)) ] ))) (defn window-hint [window] (if (= window ":all-time") "All time" (pretty-uptime-sec window))) (defn topology-action-button [id name action command is-wait default-wait enabled] [:input {:type "button" :value action (if enabled :enabled :disabled) "" :onclick (str "confirmAction('" (StringEscapeUtils/escapeJavaScript id) "', '" (StringEscapeUtils/escapeJavaScript name) "', '" command "', " is-wait ", " default-wait ")")}]) (defn topology-page [id window include-sys?] (with-nimbus nimbus (let [window (if window window ":all-time") window-hint (window-hint window) summ (.getTopologyInfo ^Nimbus$Client nimbus id) topology (.getTopology ^Nimbus$Client nimbus id) topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id)) spout-summs (filter (partial spout-summary? topology) (.get_executors summ)) bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ)) spout-comp-summs (group-by-comp spout-summs) bolt-comp-summs (group-by-comp bolt-summs) bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs) name (.get_name summ) status (.get_status summ) msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS) ] (concat [[:h2 "Topology summary"]] [(topology-summary-table summ)] [[:h2 {:class "js-only"} "Topology actions"]] [[:p {:class "js-only"} (concat [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))] [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))] [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))] [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))] )]] [[:h2 "Topology stats"]] (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?)) [[:h2 "Spouts (" window-hint ")"]] (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?) [[:h2 "Bolts (" window-hint ")"]] (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?) [[:h2 "Topology Configuration"]] (configuration-table topology-conf) )))) (defn component-task-summs [^TopologyInfo summ topology id] (let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ)) bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ)) spout-comp-summs (group-by-comp spout-summs) bolt-comp-summs (group-by-comp bolt-summs) ret (if (contains? spout-comp-summs id) (spout-comp-summs id) (bolt-comp-summs id))] (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret) )) (defn spout-summary-table [topology-id id stats window] (let [times (stats-times (:emitted stats)) display-map (into {} (for [t times] [t pretty-uptime-sec])) display-map (assoc display-map ":all-time" (fn [_] "All time"))] (sorted-table ["Window" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed"] (for [k (concat times [":all-time"]) :let [disp ((display-map k) k)]] [(link-to (if (= k window) {:class "red"} {}) (url-format "/topology/%s/component/%s?window=%s" topology-id id k) disp) (get-in stats [:emitted k]) (get-in stats [:transferred k]) (float-str (get-in stats [:complete-latencies k])) (get-in stats [:acked k]) (get-in stats [:failed k]) ]) :time-cols [0]))) (defn spout-output-summary-table [stream-summary window] (let [stream-summary (map-val swap-map-order (swap-map-order stream-summary))] (sorted-table ["Stream" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed"] (for [[s stats] (stream-summary window)] [s (nil-to-zero (:emitted stats)) (nil-to-zero (:transferred stats)) (float-str (:complete-latencies stats)) (nil-to-zero (:acked stats)) (nil-to-zero (:failed stats))]) ))) (defn spout-executor-table [topology-id executors window include-sys?] (sorted-table ["Id" "Uptime" "Host" "Port" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed"] (for [^ExecutorSummary e executors :let [stats (.get_stats e) stats (if stats (-> stats (aggregate-spout-stats include-sys?) aggregate-spout-streams swap-map-order (get window)))]] [(pretty-executor-info (.get_executor_info e)) (pretty-uptime-sec (.get_uptime_secs e)) (.get_host e) (worker-log-link (.get_host e) (.get_port e)) (nil-to-zero (:emitted stats)) (nil-to-zero (:transferred stats)) (float-str (:complete-latencies stats)) (nil-to-zero (:acked stats)) (nil-to-zero (:failed stats)) ] ) :time-cols [1] )) (defn spout-page [window ^TopologyInfo topology-info component executors include-sys?] (let [window-hint (str " (" (window-hint window) ")") stats (get-filled-stats executors) stream-summary (-> stats (aggregate-spout-stats include-sys?)) summary (-> stream-summary aggregate-spout-streams)] (concat [[:h2 "Spout stats"]] (spout-summary-table (.get_id topology-info) component summary window) [[:h2 "Output stats" window-hint]] (spout-output-summary-table stream-summary window) [[:h2 "Executors" window-hint]] (spout-executor-table (.get_id topology-info) executors window include-sys?) ;; task id, task uptime, stream aggregated stats, last error ))) (defn bolt-output-summary-table [stream-summary window] (let [stream-summary (-> stream-summary swap-map-order (get window) (select-keys [:emitted :transferred]) swap-map-order)] (sorted-table ["Stream" "Emitted" "Transferred"] (for [[s stats] stream-summary] [s (nil-to-zero (:emitted stats)) (nil-to-zero (:transferred stats)) ]) ))) (defn bolt-input-summary-table [stream-summary window] (let [stream-summary (-> stream-summary swap-map-order (get window) (select-keys [:acked :failed :process-latencies :executed :execute-latencies]) swap-map-order)] (sorted-table ["Component" "Stream" "Execute latency (ms)" "Executed" "Process latency (ms)" "Acked" "Failed"] (for [[^GlobalStreamId s stats] stream-summary] [(escape-html (.get_componentId s)) (.get_streamId s) (float-str (:execute-latencies stats)) (nil-to-zero (:executed stats)) (float-str (:process-latencies stats)) (nil-to-zero (:acked stats)) (nil-to-zero (:failed stats)) ]) ))) (defn bolt-executor-table [topology-id executors window include-sys?] (sorted-table ["Id" "Uptime" "Host" "Port" "Emitted" "Transferred" "Capacity (last 10m)" "Execute latency (ms)" "Executed" "Process latency (ms)" "Acked" "Failed"] (for [^ExecutorSummary e executors :let [stats (.get_stats e) stats (if stats (-> stats (aggregate-bolt-stats include-sys?) (aggregate-bolt-streams) swap-map-order (get window)))]] [(pretty-executor-info (.get_executor_info e)) (pretty-uptime-sec (.get_uptime_secs e)) (.get_host e) (worker-log-link (.get_host e) (.get_port e)) (nil-to-zero (:emitted stats)) (nil-to-zero (:transferred stats)) (render-capacity (compute-executor-capacity e)) (float-str (:execute-latencies stats)) (nil-to-zero (:executed stats)) (float-str (:process-latencies stats)) (nil-to-zero (:acked stats)) (nil-to-zero (:failed stats)) ] ) :time-cols [1] )) (defn bolt-summary-table [topology-id id stats window] (let [times (stats-times (:emitted stats)) display-map (into {} (for [t times] [t pretty-uptime-sec])) display-map (assoc display-map ":all-time" (fn [_] "All time"))] (sorted-table ["Window" "Emitted" "Transferred" "Execute latency (ms)" "Executed" "Process latency (ms)" "Acked" "Failed"] (for [k (concat times [":all-time"]) :let [disp ((display-map k) k)]] [(link-to (if (= k window) {:class "red"} {}) (url-format "/topology/%s/component/%s?window=%s" topology-id id k) disp) (get-in stats [:emitted k]) (get-in stats [:transferred k]) (float-str (get-in stats [:execute-latencies k])) (get-in stats [:executed k]) (float-str (get-in stats [:process-latencies k])) (get-in stats [:acked k]) (get-in stats [:failed k]) ]) :time-cols [0]))) (defn bolt-page [window ^TopologyInfo topology-info component executors include-sys?] (let [window-hint (str " (" (window-hint window) ")") stats (get-filled-stats executors) stream-summary (-> stats (aggregate-bolt-stats include-sys?)) summary (-> stream-summary aggregate-bolt-streams)] (concat [[:h2 "Bolt stats"]] (bolt-summary-table (.get_id topology-info) component summary window) [[:h2 "Input stats" window-hint]] (bolt-input-summary-table stream-summary window) [[:h2 "Output stats" window-hint]] (bolt-output-summary-table stream-summary window) [[:h2 "Executors"]] (bolt-executor-table (.get_id topology-info) executors window include-sys?) ))) (defn errors-table [errors-list] (let [errors (->> errors-list (sort-by #(.get_error_time_secs ^ErrorInfo %)) reverse)] (sorted-table ["Time" "Error"] (for [^ErrorInfo e errors] [(date-str (.get_error_time_secs e)) [:pre (.get_error e)]]) :sort-list "[[0,1]]" ))) (defn component-page [topology-id component window include-sys?] (with-nimbus nimbus (let [window (if window window ":all-time") summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id) topology (.getTopology ^Nimbus$Client nimbus topology-id) type (component-type topology component) summs (component-task-summs summ topology component) spec (cond (= type :spout) (spout-page window summ component summs include-sys?) (= type :bolt) (bolt-page window summ component summs include-sys?))] (concat [[:h2 "Component summary"] (table ["Id" "Topology" "Executors" "Tasks"] [[(escape-html component) (topology-link (.get_id summ) (.get_name summ)) (count summs) (sum-tasks summs) ]])] spec [[:h2 "Errors"] (errors-table (get (.get_errors summ) component))] )))) (defn get-include-sys? [cookies] (let [sys? (get cookies "sys") sys? (if (or (nil? sys?) (= "false" (:value sys?))) false true)] sys?)) (defroutes main-routes (GET "/" [:as {cookies :cookies}] (-> (main-page) ui-template)) (GET "/topology/:id" [:as {cookies :cookies} id & m] (let [include-sys? (get-include-sys? cookies)] (-> (topology-page id (:window m) include-sys?) (concat [(mk-system-toggle-button include-sys?)]) ui-template))) (GET "/topology/:id/component/:component" [:as {cookies :cookies} id component & m] (let [include-sys? (get-include-sys? cookies)] (-> (component-page id component (:window m) include-sys?) (concat [(mk-system-toggle-button include-sys?)]) ui-template))) (POST "/topology/:id/activate" [id] (with-nimbus nimbus (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg)] (.activate nimbus name) (log-message "Activating topology '" name "'"))) (resp/redirect (str "/topology/" id))) (POST "/topology/:id/deactivate" [id] (with-nimbus nimbus (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg)] (.deactivate nimbus name) (log-message "Deactivating topology '" name "'"))) (resp/redirect (str "/topology/" id))) (POST "/topology/:id/rebalance/:wait-time" [id wait-time] (with-nimbus nimbus (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg) options (RebalanceOptions.)] (.set_wait_secs options (Integer/parseInt wait-time)) (.rebalance nimbus name options) (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs"))) (resp/redirect (str "/topology/" id))) (POST "/topology/:id/kill/:wait-time" [id wait-time] (with-nimbus nimbus (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg) options (KillOptions.)] (.set_wait_secs options (Integer/parseInt wait-time)) (.killTopologyWithOpts nimbus name options) (log-message "Killing topology '" name "' with wait time: " wait-time " secs"))) (resp/redirect (str "/topology/" id))) (route/resources "/") (route/not-found "Page not found")) (defn exception->html [ex] (concat [[:h2 "Internal Server Error"]] [[:pre (let [sw (java.io.StringWriter.)] (.printStackTrace ex (java.io.PrintWriter. sw)) (.toString sw))]])) (defn catch-errors [handler] (fn [request] (try (handler request) (catch Exception ex (-> (resp/response (ui-template (exception->html ex))) (resp/status 500) (resp/content-type "text/html")) )))) (def app (handler/site (-> main-routes (wrap-reload '[backtype.storm.ui.core]) catch-errors))) (defn start-server! [] (run-jetty app {:port (Integer. (*STORM-CONF* UI-PORT)) :join? false})) (defn -main [] (start-server!)) ================================================ FILE: storm-core/src/clj/backtype/storm/ui/helpers.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.ui.helpers (:use compojure.core) (:use [hiccup core page-helpers]) (:use [clojure [string :only [join]]]) (:use [backtype.storm.util :only [uuid defnk]]) (:use [clj-time coerce format]) (:import [backtype.storm.generated ExecutorInfo ExecutorSummary]) (:require [compojure.route :as route] [compojure.handler :as handler])) (defn split-divide [val divider] [(Integer. (int (/ val divider))) (mod val divider)] ) (def PRETTY-SEC-DIVIDERS [["s" 60] ["m" 60] ["h" 24] ["d" nil]]) (def PRETTY-MS-DIVIDERS (cons ["ms" 1000] PRETTY-SEC-DIVIDERS)) (defn pretty-uptime-str* [val dividers] (let [val (if (string? val) (Integer/parseInt val) val) vals (reduce (fn [[state val] [_ divider]] (if (pos? val) (let [[divided mod] (if divider (split-divide val divider) [nil val])] [(concat state [mod]) divided] ) [state val] )) [[] val] dividers) strs (->> (first vals) (map (fn [[suffix _] val] (str val suffix)) dividers ))] (join " " (reverse strs)) )) (defn pretty-uptime-sec [secs] (pretty-uptime-str* secs PRETTY-SEC-DIVIDERS)) (defn pretty-uptime-ms [ms] (pretty-uptime-str* ms PRETTY-MS-DIVIDERS)) (defelem table [headers data] [:table [:thead [:tr (for [h headers] [:th h]) ]] [:tbody (for [row data] [:tr (for [col row] [:td col] )] )] ]) (defnk sort-table [id :sort-list "[[0,0]]" :time-cols []] (let [strs (for [c time-cols] (format "%s: { sorter: 'stormtimestr'}" c)) sorters (join ", " strs)] [:script (format "$(document).ready(function() { $(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers: {%s}}); }); });" id sort-list sorters)])) (defn float-str [n] (if n (format "%.3f" (float n)) "0" )) (defn swap-map-order [m] (->> m (map (fn [[k v]] (into {} (for [[k2 v2] v] [k2 {k v2}] )) )) (apply merge-with merge) )) (defn sorted-table [headers data & args] (let [id (uuid)] (concat [(table {:class "zebra-striped" :id id} headers data)] (if-not (empty? data) [(apply sort-table id args)]) ))) (defn date-str [secs] (let [dt (from-long (* 1000 (long secs)))] (unparse (:rfc822 formatters) dt) )) (defn url-format [fmt & args] (String/format fmt (to-array (map #(java.net.URLEncoder/encode (str %)) args)))) (defn to-tasks [^ExecutorInfo e] (let [start (.get_task_start e) end (.get_task_end e)] (range start (inc end)) )) (defn sum-tasks [executors] (reduce + (->> executors (map #(.get_executor_info ^ExecutorSummary %)) (map to-tasks) (map count)))) (defn pretty-executor-info [^ExecutorInfo e] (str "[" (.get_task_start e) "-" (.get_task_end e) "]")) ================================================ FILE: storm-core/src/clj/backtype/storm/util.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.util (:import [java.net InetAddress]) (:import [java.util Map Map$Entry List ArrayList Collection Iterator HashMap]) (:import [java.io FileReader FileNotFoundException]) (:import [backtype.storm Config]) (:import [backtype.storm.utils Time Container ClojureTimerTask Utils MutableObject MutableInt]) (:import [java.util UUID Random ArrayList List Collections]) (:import [java.util.zip ZipFile]) (:import [java.util.concurrent.locks ReentrantReadWriteLock]) (:import [java.util.concurrent Semaphore]) (:import [java.io File RandomAccessFile StringWriter PrintWriter]) (:import [java.lang.management ManagementFactory]) (:import [org.apache.commons.exec DefaultExecutor CommandLine]) (:import [org.apache.commons.io FileUtils]) (:import [org.apache.commons.exec ExecuteException]) (:import [org.json.simple JSONValue]) (:require [clojure [string :as str]]) (:import [clojure.lang RT]) (:require [clojure [set :as set]]) (:use [clojure walk]) (:use [backtype.storm log]) ) (defn wrap-in-runtime "Wraps an exception in a RuntimeException if needed" [^Exception e] (if (instance? RuntimeException e) e (RuntimeException. e))) (defmacro defalias "Defines an alias for a var: a new var with the same root binding (if any) and similar metadata. The metadata of the alias is its initial metadata (as provided by def) merged into the metadata of the original." ([name orig] `(do (alter-meta! (if (.hasRoot (var ~orig)) (def ~name (.getRawRoot (var ~orig))) (def ~name)) ;; When copying metadata, disregard {:macro false}. ;; Workaround for http://www.assembla.com/spaces/clojure/tickets/273 #(conj (dissoc % :macro) (apply dissoc (meta (var ~orig)) (remove #{:macro} (keys %))))) (var ~name))) ([name orig doc] (list `defalias (with-meta name (assoc (meta name) :doc doc)) orig))) ;; name-with-attributes by Konrad Hinsen: (defn name-with-attributes "To be used in macro definitions. Handles optional docstrings and attribute maps for a name to be defined in a list of macro arguments. If the first macro argument is a string, it is added as a docstring to name and removed from the macro argument list. If afterwards the first macro argument is a map, its entries are added to the name's metadata map and the map is removed from the macro argument list. The return value is a vector containing the name with its extended metadata map and the list of unprocessed macro arguments." [name macro-args] (let [[docstring macro-args] (if (string? (first macro-args)) [(first macro-args) (next macro-args)] [nil macro-args]) [attr macro-args] (if (map? (first macro-args)) [(first macro-args) (next macro-args)] [{} macro-args]) attr (if docstring (assoc attr :doc docstring) attr) attr (if (meta name) (conj (meta name) attr) attr)] [(with-meta name attr) macro-args])) (defmacro defnk "Define a function accepting keyword arguments. Symbols up to the first keyword in the parameter list are taken as positional arguments. Then an alternating sequence of keywords and defaults values is expected. The values of the keyword arguments are available in the function body by virtue of the symbol corresponding to the keyword (cf. :keys destructuring). defnk accepts an optional docstring as well as an optional metadata map." [fn-name & fn-tail] (let [[fn-name [args & body]] (name-with-attributes fn-name fn-tail) [pos kw-vals] (split-with symbol? args) syms (map #(-> % name symbol) (take-nth 2 kw-vals)) values (take-nth 2 (rest kw-vals)) sym-vals (apply hash-map (interleave syms values)) de-map {:keys (vec syms) :or sym-vals}] `(defn ~fn-name [~@pos & options#] (let [~de-map (apply hash-map options#)] ~@body)))) (defn find-first "Returns the first item of coll for which (pred item) returns logical true. Consumes sequences up to the first match, will consume the entire sequence and return nil if no match is found." [pred coll] (first (filter pred coll))) (defn dissoc-in "Dissociates an entry from a nested associative structure returning a new nested structure. keys is a sequence of keys. Any empty maps that result will not be present in the new structure." [m [k & ks :as keys]] (if ks (if-let [nextmap (get m k)] (let [newmap (dissoc-in nextmap ks)] (if (seq newmap) (assoc m k newmap) (dissoc m k))) m) (dissoc m k))) (defn indexed "Returns a lazy sequence of [index, item] pairs, where items come from 's' and indexes count up from zero. (indexed '(a b c d)) => ([0 a] [1 b] [2 c] [3 d])" [s] (map vector (iterate inc 0) s)) (defn positions "Returns a lazy sequence containing the positions at which pred is true for items in coll." [pred coll] (for [[idx elt] (indexed coll) :when (pred elt)] idx)) (defn exception-cause? [klass ^Throwable t] (->> (iterate #(.getCause ^Throwable %) t) (take-while identity) (some (partial instance? klass)) boolean)) (defmacro thrown-cause? [klass & body] `(try ~@body false (catch Throwable t# (exception-cause? ~klass t#)))) (defmacro thrown-cause-with-msg? [klass re & body] `(try ~@body false (catch Throwable t# (and (re-matches ~re (.getMessage t#)) (exception-cause? ~klass t#))))) (defmacro forcat [[args aseq] & body] `(mapcat (fn [~args] ~@body) ~aseq)) (defmacro try-cause [& body] (let [checker (fn [form] (or (not (sequential? form)) (not= 'catch (first form)))) [code guards] (split-with checker body) error-local (gensym "t") guards (forcat [[_ klass local & guard-body] guards] `((exception-cause? ~klass ~error-local) (let [~local ~error-local] ~@guard-body ))) ] `(try ~@code (catch Throwable ~error-local (cond ~@guards true (throw ~error-local) ))))) (defn local-hostname [] (.getCanonicalHostName (InetAddress/getLocalHost))) (letfn [(try-port [port] (with-open [socket (java.net.ServerSocket. port)] (.getLocalPort socket)))] (defn available-port ([] (try-port 0)) ([preferred] (try (try-port preferred) (catch java.io.IOException e (available-port)))))) (defn uuid [] (str (UUID/randomUUID))) (defn current-time-secs [] (Time/currentTimeSecs)) (defn current-time-millis [] (Time/currentTimeMillis)) (defn clojurify-structure [s] (prewalk (fn [x] (cond (instance? Map x) (into {} x) (instance? List x) (vec x) true x)) s)) (defmacro with-file-lock [path & body] `(let [f# (File. ~path) _# (.createNewFile f#) rf# (RandomAccessFile. f# "rw") lock# (.. rf# (getChannel) (lock))] (try ~@body (finally (.release lock#) (.close rf#)) ))) (defn tokenize-path [^String path] (let [toks (.split path "/")] (vec (filter (complement empty?) toks)) )) (defn assoc-conj [m k v] (merge-with concat m {k [v]})) ;; returns [ones in first set not in second, ones in second set not in first] (defn set-delta [old curr] (let [s1 (set old) s2 (set curr)] [(set/difference s1 s2) (set/difference s2 s1)] )) (defn parent-path [path] (let [toks (tokenize-path path)] (str "/" (str/join "/" (butlast toks))) )) (defn toks->path [toks] (str "/" (str/join "/" toks)) ) (defn normalize-path [^String path] (toks->path (tokenize-path path))) (defn map-val [afn amap] (into {} (for [[k v] amap] [k (afn v)] ))) (defn filter-val [afn amap] (into {} (filter (fn [[k v]] (afn v)) amap ))) (defn filter-key [afn amap] (into {} (filter (fn [[k v]] (afn k)) amap ))) (defn map-key [afn amap] (into {} (for [[k v] amap] [(afn k) v] ))) (defn separate [pred aseq] [(filter pred aseq) (filter (complement pred) aseq)]) (defn full-path [parent name] (let [toks (tokenize-path parent)] (toks->path (conj toks name)) )) (def not-nil? (complement nil?)) (defn barr [& vals] (byte-array (map byte vals))) (defn halt-process! [val & msg] (log-message "Halting process: " msg) (.halt (Runtime/getRuntime) val) ) (defn sum [vals] (reduce + vals)) (defn repeat-seq ([aseq] (apply concat (repeat aseq))) ([amt aseq] (apply concat (repeat amt aseq)) )) (defn div "Perform floating point division on the arguments." [f & rest] (apply / (double f) rest)) (defn defaulted [val default] (if val val default)) (defn mk-counter ([] (mk-counter 1)) ([start-val] (let [val (atom (dec start-val))] (fn [] (swap! val inc))))) (defmacro for-times [times & body] `(for [i# (range ~times)] ~@body )) (defmacro dofor [& body] `(doall (for ~@body))) (defn reverse-map "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}" [amap] (reduce (fn [m [k v]] (let [existing (get m v [])] (assoc m v (conj existing k)))) {} amap)) (defmacro print-vars [& vars] (let [prints (for [v vars] `(println ~(str v) ~v))] `(do ~@prints))) (defn process-pid "Gets the pid of this JVM. Hacky because Java doesn't provide a real way to do this." [] (let [name (.getName (ManagementFactory/getRuntimeMXBean)) split (.split name "@")] (when-not (= 2 (count split)) (throw (RuntimeException. (str "Got unexpected process name: " name)))) (first split) )) (defn exec-command! [command] (let [[comm-str & args] (seq (.split command " ")) command (CommandLine. comm-str)] (doseq [a args] (.addArgument command a)) (.execute (DefaultExecutor.) command) )) (defn extract-dir-from-jar [jarpath dir destdir] (try-cause (exec-command! (str "unzip -qq " jarpath " " dir "/** -d " destdir)) (catch ExecuteException e (log-message "Could not extract " dir " from " jarpath)) )) (defn ensure-process-killed! [pid] ;; TODO: should probably do a ps ax of some sort to make sure it was killed (try-cause (exec-command! (str "kill -9 " pid)) (catch ExecuteException e (log-message "Error when trying to kill " pid ". Process is probably already dead.")) )) (defnk launch-process [command :environment {}] (let [command (->> (seq (.split command " ")) (filter (complement empty?))) builder (ProcessBuilder. command) process-env (.environment builder)] (doseq [[k v] environment] (.put process-env k v)) (.start builder) )) (defn sleep-secs [secs] (when (pos? secs) (Time/sleep (* (long secs) 1000)))) (defn sleep-until-secs [target-secs] (Time/sleepUntil (* (long target-secs) 1000))) (defprotocol SmartThread (start [this]) (join [this]) (interrupt [this]) (sleeping? [this])) ;; afn returns amount of time to sleep (defnk async-loop [afn :daemon false :kill-fn (fn [error] (halt-process! 1 "Async loop died!")) :priority Thread/NORM_PRIORITY :factory? false :start true :thread-name nil] (let [thread (Thread. (fn [] (try-cause (let [afn (if factory? (afn) afn)] (loop [] (let [sleep-time (afn)] (when-not (nil? sleep-time) (sleep-secs sleep-time) (recur)) ))) (catch InterruptedException e (log-message "Async loop interrupted!") ) (catch Throwable t (log-error t "Async loop died!") (kill-fn t) )) ))] (.setDaemon thread daemon) (.setPriority thread priority) (when thread-name (.setName thread (str (.getName thread) "-" thread-name))) (when start (.start thread)) ;; should return object that supports stop, interrupt, join, and waiting? (reify SmartThread (start [this] (.start thread)) (join [this] (.join thread)) (interrupt [this] (.interrupt thread)) (sleeping? [this] (Time/isThreadWaiting thread) )) )) (defn exists-file? [path] (.exists (File. path))) (defn rmr [path] (log-debug "Rmr path " path) (when (exists-file? path) (try (FileUtils/forceDelete (File. path)) (catch FileNotFoundException e)))) (defn rmpath "Removes file or directory at the path. Not recursive. Throws exception on failure" [path] (log-debug "Removing path " path) (when (exists-file? path) (let [deleted? (.delete (File. path))] (when-not deleted? (throw (RuntimeException. (str "Failed to delete " path)))) ))) (defn local-mkdirs [path] (log-debug "Making dirs at " path) (FileUtils/forceMkdir (File. path))) (defn touch [path] (log-debug "Touching file at " path) (let [success? (.createNewFile (File. path))] (when-not success? (throw (RuntimeException. (str "Failed to touch " path)))) )) (defn read-dir-contents [dir] (if (exists-file? dir) (let [content-files (.listFiles (File. dir))] (map #(.getName ^File %) content-files)) [] )) (defn compact [aseq] (filter (complement nil?) aseq)) (defn current-classpath [] (System/getProperty "java.class.path")) (defn add-to-classpath [classpath paths] (str/join ":" (cons classpath paths))) (defn ^ReentrantReadWriteLock mk-rw-lock [] (ReentrantReadWriteLock.)) (defmacro read-locked [rw-lock & body] (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})] `(let [rlock# (.readLock ~lock)] (try (.lock rlock#) ~@body (finally (.unlock rlock#)))))) (defmacro write-locked [rw-lock & body] (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})] `(let [wlock# (.writeLock ~lock)] (try (.lock wlock#) ~@body (finally (.unlock wlock#)))))) (defn wait-for-condition [apredicate] (while (not (apredicate)) (Time/sleep 100) )) (defn some? [pred aseq] ((complement nil?) (some pred aseq))) (defn time-delta [time-secs] (- (current-time-secs) time-secs)) (defn time-delta-ms [time-ms] (- (System/currentTimeMillis) (long time-ms))) (defn parse-int [str] (Integer/valueOf str)) (defn integer-divided [sum num-pieces] (clojurify-structure (Utils/integerDivided sum num-pieces))) (defn collectify [obj] (if (or (sequential? obj) (instance? Collection obj)) obj [obj])) (defn to-json [obj] (JSONValue/toJSONString obj)) (defn from-json [^String str] (if str (clojurify-structure (JSONValue/parse str)) nil )) (defmacro letlocals [& body] (let [[tobind lexpr] (split-at (dec (count body)) body) binded (vec (mapcat (fn [e] (if (and (list? e) (= 'bind (first e))) [(second e) (last e)] ['_ e] )) tobind ))] `(let ~binded ~(first lexpr) ))) (defn remove-first [pred aseq] (let [[b e] (split-with (complement pred) aseq)] (when (empty? e) (throw (IllegalArgumentException. "Nothing to remove"))) (concat b (rest e)) )) (defn assoc-non-nil [m k v] (if v (assoc m k v) m)) (defn multi-set "Returns a map of elem to count" [aseq] (apply merge-with + (map #(hash-map % 1) aseq))) (defn set-var-root* [avar val] (alter-var-root avar (fn [avar] val))) (defmacro set-var-root [var-sym val] `(set-var-root* (var ~var-sym) ~val)) (defmacro with-var-roots [bindings & body] (let [settings (partition 2 bindings) tmpvars (repeatedly (count settings) (partial gensym "old")) vars (map first settings) savevals (vec (mapcat (fn [t v] [t v]) tmpvars vars)) setters (for [[v s] settings] `(set-var-root ~v ~s)) restorers (map (fn [v s] `(set-var-root ~v ~s)) vars tmpvars) ] `(let ~savevals ~@setters (try ~@body (finally ~@restorers)) ))) (defn map-diff "Returns mappings in m2 that aren't in m1" [m1 m2] (into {} (filter (fn [[k v]] (not= v (m1 k))) m2 ))) (defn select-keys-pred [pred amap] (into {} (filter (fn [[k v]] (pred k)) amap))) (defn rotating-random-range [choices] (let [rand (Random.) choices (ArrayList. choices)] (Collections/shuffle choices rand) [(MutableInt. -1) choices rand])) (defn acquire-random-range-id [[^MutableInt curr ^List state ^Random rand]] (when (>= (.increment curr) (.size state)) (.set curr 0) (Collections/shuffle state rand)) (.get state (.get curr))) ; this can be rewritten to be tail recursive (defn interleave-all [& colls] (if (empty? colls) [] (let [colls (filter (complement empty?) colls) my-elems (map first colls) rest-elems (apply interleave-all (map rest colls))] (concat my-elems rest-elems) ))) (defn update [m k afn] (assoc m k (afn (get m k)))) (defn any-intersection [& sets] (let [elem->count (multi-set (apply concat sets))] (-> (filter-val #(> % 1) elem->count) keys ))) (defn between? "val >= lower and val <= upper" [val lower upper] (and (>= val lower) (<= val upper))) (defmacro benchmark [& body] `(let [l# (doall (range 1000000))] (time (doseq [i# l#] ~@body)))) (defn rand-sampler [freq] (let [r (java.util.Random.)] (fn [] (= 0 (.nextInt r freq))) )) (defn even-sampler [freq] (let [freq (int freq) start (int 0) r (java.util.Random.) curr (MutableInt. -1) target (MutableInt. (.nextInt r freq))] (with-meta (fn [] (let [i (.increment curr)] (when (>= i freq) (.set curr start) (.set target (.nextInt r freq)))) (= (.get curr) (.get target))) {:rate freq}))) (defn sampler-rate [sampler] (:rate (meta sampler))) (defn class-selector [obj & args] (class obj)) (defn uptime-computer [] (let [start-time (current-time-secs)] (fn [] (time-delta start-time) ))) (defn stringify-error [error] (let [result (StringWriter.) printer (PrintWriter. result)] (.printStackTrace error printer) (.toString result) )) (defn nil-to-zero [v] (or v 0)) (defn bit-xor-vals [vals] (reduce bit-xor 0 vals)) (defmacro with-error-reaction [afn & body] `(try ~@body (catch Throwable t# (~afn t#)))) (defn container [] (Container.)) (defn container-set! [^Container container obj] (set! (. container object) obj) container) (defn container-get [^Container container] (. container object)) (defn to-millis [secs] (* 1000 (long secs))) (defn throw-runtime [& strs] (throw (RuntimeException. (apply str strs)))) (defn redirect-stdio-to-slf4j! [] ;; set-var-root doesn't work with *out* and *err*, so digging much deeper here ;; Unfortunately, this code seems to work at the REPL but not when spawned as worker processes ;; it might have something to do with being a child process ;; (set! (. (.getThreadBinding RT/OUT) val) ;; (java.io.OutputStreamWriter. ;; (log-stream :info "STDIO"))) ;; (set! (. (.getThreadBinding RT/ERR) val) ;; (PrintWriter. ;; (java.io.OutputStreamWriter. ;; (log-stream :error "STDIO")) ;; true)) (log-capture! "STDIO")) (defn spy [prefix val] (log-message prefix ": " val) val) (defn zip-contains-dir? [zipfile target] (let [entries (->> zipfile (ZipFile.) .entries enumeration-seq (map (memfn getName)))] (some? #(.startsWith % (str target "/")) entries) )) (defn url-encode [s] (java.net.URLEncoder/encode s)) (defn join-maps [& maps] (let [all-keys (apply set/union (for [m maps] (-> m keys set)))] (into {} (for [k all-keys] [k (for [m maps] (m k))] )))) (defn partition-fixed [max-num-chunks aseq] (if (zero? max-num-chunks) [] (let [chunks (->> (integer-divided (count aseq) max-num-chunks) (#(dissoc % 0)) (sort-by (comp - first)) (mapcat (fn [[size amt]] (repeat amt size))) )] (loop [result [] [chunk & rest-chunks] chunks data aseq] (if (nil? chunk) result (let [[c rest-data] (split-at chunk data)] (recur (conj result c) rest-chunks rest-data))))))) (defn assoc-apply-self [curr key afn] (assoc curr key (afn curr))) (defmacro recursive-map [& forms] (->> (partition 2 forms) (map (fn [[key form]] `(assoc-apply-self ~key (fn [~'<>] ~form)))) (concat `(-> {})))) (defn current-stack-trace [] (->> (Thread/currentThread) .getStackTrace (map str) (str/join "\n") )) (defn get-iterator [^Iterable alist] (if alist (.iterator alist))) (defn iter-has-next? [^Iterator iter] (if iter (.hasNext iter) false)) (defn iter-next [^Iterator iter] (.next iter)) (defmacro fast-list-iter [pairs & body] (let [pairs (partition 2 pairs) lists (map second pairs) elems (map first pairs) iters (map (fn [_] (gensym)) lists) bindings (->> (map (fn [i l] [i `(get-iterator ~l)]) iters lists) (apply concat)) tests (map (fn [i] `(iter-has-next? ~i)) iters) assignments (->> (map (fn [e i] [e `(iter-next ~i)]) elems iters) (apply concat))] `(let [~@bindings] (while (and ~@tests) (let [~@assignments] ~@body ))))) (defn fast-list-map [afn alist] (let [ret (ArrayList.)] (fast-list-iter [e alist] (.add ret (afn e))) ret )) (defmacro fast-list-for [[e alist] & body] `(fast-list-map (fn [~e] ~@body) ~alist)) (defn map-iter [^Map amap] (if amap (-> amap .entrySet .iterator))) (defn convert-entry [^Map$Entry entry] [(.getKey entry) (.getValue entry)]) (defmacro fast-map-iter [[bind amap] & body] `(let [iter# (map-iter ~amap)] (while (iter-has-next? iter#) (let [entry# (iter-next iter#) ~bind (convert-entry entry#)] ~@body )))) (defn fast-first [^List alist] (.get alist 0)) (defmacro get-with-default [amap key default-val] `(let [curr# (.get ~amap ~key)] (if curr# curr# (do (let [new# ~default-val] (.put ~amap ~key new#) new# ))))) (defn fast-group-by [afn alist] (let [ret (HashMap.)] (fast-list-iter [e alist] (let [key (afn e) ^List curr (get-with-default ret key (ArrayList.))] (.add curr e))) ret )) (defn new-instance [klass] (let [klass (if (string? klass) (Class/forName klass) klass)] (.newInstance klass) )) (defmacro -<> ([x] x) ([x form] (if (seq? form) (with-meta (let [[begin [_ & end]] (split-with #(not= % '<>) form)] (concat begin [x] end)) (meta form)) (list form x))) ([x form & more] `(-<> (-<> ~x ~form) ~@more))) ================================================ FILE: storm-core/src/clj/backtype/storm/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.zookeeper (:import [com.netflix.curator.retry RetryNTimes]) (:import [com.netflix.curator.framework.api CuratorEvent CuratorEventType CuratorListener UnhandledErrorListener]) (:import [com.netflix.curator.framework CuratorFramework CuratorFrameworkFactory]) (:import [org.apache.zookeeper ZooKeeper Watcher KeeperException$NoNodeException ZooDefs ZooDefs$Ids CreateMode WatchedEvent Watcher$Event Watcher$Event$KeeperState Watcher$Event$EventType KeeperException$NodeExistsException]) (:import [org.apache.zookeeper.data Stat]) (:import [org.apache.zookeeper.server ZooKeeperServer NIOServerCnxn$Factory]) (:import [java.net InetSocketAddress BindException]) (:import [java.io File]) (:import [backtype.storm.utils Utils ZookeeperAuthInfo]) (:use [backtype.storm util log config])) (def zk-keeper-states {Watcher$Event$KeeperState/Disconnected :disconnected Watcher$Event$KeeperState/SyncConnected :connected Watcher$Event$KeeperState/AuthFailed :auth-failed Watcher$Event$KeeperState/Expired :expired }) (def zk-event-types {Watcher$Event$EventType/None :none Watcher$Event$EventType/NodeCreated :node-created Watcher$Event$EventType/NodeDeleted :node-deleted Watcher$Event$EventType/NodeDataChanged :node-data-changed Watcher$Event$EventType/NodeChildrenChanged :node-children-changed }) (defn- default-watcher [state type path] (log-message "Zookeeper state update: " state type path)) (defnk mk-client [conf servers port :root "" :watcher default-watcher :auth-conf nil] (let [fk (Utils/newCurator conf servers port root (when auth-conf (ZookeeperAuthInfo. auth-conf)))] (.. fk (getCuratorListenable) (addListener (reify CuratorListener (^void eventReceived [this ^CuratorFramework _fk ^CuratorEvent e] (when (= (.getType e) CuratorEventType/WATCHED) (let [^WatchedEvent event (.getWatchedEvent e)] (watcher (zk-keeper-states (.getState event)) (zk-event-types (.getType event)) (.getPath event)))))))) ;; (.. fk ;; (getUnhandledErrorListenable) ;; (addListener ;; (reify UnhandledErrorListener ;; (unhandledError [this msg error] ;; (if (or (exception-cause? InterruptedException error) ;; (exception-cause? java.nio.channels.ClosedByInterruptException error)) ;; (do (log-warn-error error "Zookeeper exception " msg) ;; (let [to-throw (InterruptedException.)] ;; (.initCause to-throw error) ;; (throw to-throw) ;; )) ;; (do (log-error error "Unrecoverable Zookeeper error " msg) ;; (halt-process! 1 "Unrecoverable Zookeeper error"))) ;; )))) (.start fk) fk)) (def zk-create-modes {:ephemeral CreateMode/EPHEMERAL :persistent CreateMode/PERSISTENT :sequential CreateMode/PERSISTENT_SEQUENTIAL}) (defn create-node ([^CuratorFramework zk ^String path ^bytes data mode] (try (.. zk (create) (withMode (zk-create-modes mode)) (withACL ZooDefs$Ids/OPEN_ACL_UNSAFE) (forPath (normalize-path path) data)) (catch Exception e (throw (wrap-in-runtime e))))) ([^CuratorFramework zk ^String path ^bytes data] (create-node zk path data :persistent))) (defn exists-node? [^CuratorFramework zk ^String path watch?] ((complement nil?) (try (if watch? (.. zk (checkExists) (watched) (forPath (normalize-path path))) (.. zk (checkExists) (forPath (normalize-path path)))) (catch Exception e (throw (wrap-in-runtime e)))))) (defnk delete-node [^CuratorFramework zk ^String path :force false] (try-cause (.. zk (delete) (forPath (normalize-path path))) (catch KeeperException$NoNodeException e (when-not force (throw e))) (catch Exception e (throw (wrap-in-runtime e))))) (defn mkdirs [^CuratorFramework zk ^String path] (let [path (normalize-path path)] (when-not (or (= path "/") (exists-node? zk path false)) (mkdirs zk (parent-path path)) (try-cause (create-node zk path (barr 7) :persistent) (catch KeeperException$NodeExistsException e ;; this can happen when multiple clients doing mkdir at same time )) ))) (defn get-data [^CuratorFramework zk ^String path watch?] (let [path (normalize-path path)] (try-cause (if (exists-node? zk path watch?) (if watch? (.. zk (getData) (watched) (forPath path)) (.. zk (getData) (forPath path)))) (catch KeeperException$NoNodeException e ;; this is fine b/c we still have a watch from the successful exists call nil ) (catch Exception e (throw (wrap-in-runtime e)))))) (defn get-children [^CuratorFramework zk ^String path watch?] (try (if watch? (.. zk (getChildren) (watched) (forPath (normalize-path path))) (.. zk (getChildren) (forPath (normalize-path path)))) (catch Exception e (throw (wrap-in-runtime e))))) (defn set-data [^CuratorFramework zk ^String path ^bytes data] (try (.. zk (setData) (forPath (normalize-path path) data)) (catch Exception e (throw (wrap-in-runtime e))))) (defn exists [^CuratorFramework zk ^String path watch?] (exists-node? zk path watch?)) (defn delete-recursive [^CuratorFramework zk ^String path] (let [path (normalize-path path)] (when (exists-node? zk path false) (let [children (try-cause (get-children zk path false) (catch KeeperException$NoNodeException e [] ))] (doseq [c children] (delete-recursive zk (full-path path c))) (delete-node zk path :force true) )))) (defnk mk-inprocess-zookeeper [localdir :port nil] (let [localfile (File. localdir) zk (ZooKeeperServer. localfile localfile 2000) [retport factory] (loop [retport (if port port 2000)] (if-let [factory-tmp (try-cause (NIOServerCnxn$Factory. (InetSocketAddress. retport)) (catch BindException e (when (> (inc retport) (if port port 65535)) (throw (RuntimeException. "No port is available to launch an inprocess zookeeper.")))))] [retport factory-tmp] (recur (inc retport))))] (log-message "Starting inprocess zookeeper at port " retport " and dir " localdir) (.startup factory zk) [retport factory] )) (defn shutdown-inprocess-zookeeper [handle] (.shutdown handle)) ================================================ FILE: storm-core/src/clj/storm/trident/testing.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 storm.trident.testing (:import [storm.trident.testing FeederBatchSpout FeederCommitterBatchSpout MemoryMapState MemoryMapState$Factory TuplifyArgs]) (:import [backtype.storm LocalDRPC]) (:import [backtype.storm.tuple Fields]) (:import [backtype.storm.generated KillOptions]) (:require [backtype.storm [testing :as t]]) (:use [backtype.storm util]) ) (defn local-drpc [] (LocalDRPC.)) (defn exec-drpc [^LocalDRPC drpc function-name args] (let [res (.execute drpc function-name args)] (from-json res))) (defn exec-drpc-tuples [^LocalDRPC drpc function-name tuples] (exec-drpc drpc function-name (to-json tuples))) (defn feeder-spout [fields] (FeederBatchSpout. fields)) (defn feeder-committer-spout [fields] (FeederCommitterBatchSpout. fields)) (defn feed [feeder tuples] (.feed feeder tuples)) (defn fields [& fields] (Fields. fields)) (defn memory-map-state [] (MemoryMapState$Factory.)) (defmacro with-drpc [[drpc] & body] `(let [~drpc (backtype.storm.LocalDRPC.)] ~@body (.shutdown ~drpc) )) (defn with-topology* [cluster topo body-fn] (t/submit-local-topology (:nimbus cluster) "tester" {} (.build topo)) (body-fn) (.killTopologyWithOpts (:nimbus cluster) "tester" (doto (KillOptions.) (.set_wait_secs 0))) ) (defmacro with-topology [[cluster topo] & body] `(with-topology* ~cluster ~topo (fn [] ~@body))) (defn bootstrap-imports [] (import 'backtype.storm.LocalDRPC) (import 'storm.trident.TridentTopology) (import '[storm.trident.operation.builtin Count Sum Equals MapGet Debug FilterNull FirstN TupleCollectionGet]) ) (defn drpc-tuples-input [topology function-name drpc outfields] (-> topology (.newDRPCStream function-name drpc) (.each (fields "args") (TuplifyArgs.) outfields) )) ================================================ FILE: storm-core/src/clj/zilch/mq.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. ;; Copyright 2011 Tim Dysinger ;; 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. (ns zilch.mq (:refer-clojure :exclude [send]) ) (defmacro zeromq-imports [] '(do (import '[org.zeromq ZMQ ZMQ$Context ZMQ$Socket]) )) (zeromq-imports) (defn ^ZMQ$Context context [threads] (ZMQ/context threads)) (defmacro with-context [id threads & body] `(let [~id (context ~threads)] (try ~@body (finally (.term ~id))))) (def sndmore ZMQ/SNDMORE) (def req ZMQ/REQ) (def rep ZMQ/REP) (def xreq ZMQ/XREQ) (def xrep ZMQ/XREP) (def pub ZMQ/PUB) (def sub ZMQ/SUB) (def pair ZMQ/PAIR) (def push ZMQ/PUSH) (def pull ZMQ/PULL) (defn ^bytes barr [& arr] (byte-array (map byte arr))) (defn ^ZMQ$Socket socket [^ZMQ$Context context type] (.socket context type)) (defn set-linger [^ZMQ$Socket socket linger-ms] (doto socket (.setLinger (long linger-ms)))) (defn set-hwm [^ZMQ$Socket socket hwm] (if hwm (doto socket (.setHWM (long hwm))) socket )) (defn bind [^ZMQ$Socket socket url] (doto socket (.bind url))) (defn connect [^ZMQ$Socket socket url] (doto socket (.connect url))) (defn subscribe ([^ZMQ$Socket socket ^bytes topic] (doto socket (.subscribe topic))) ([^ZMQ$Socket socket] (subscribe socket (byte-array [])))) (defn unsubscribe ([^ZMQ$Socket socket ^bytes topic] (doto socket (.unsubscribe (.getBytes topic)))) ([^ZMQ$Socket socket] (unsubscribe socket ""))) (defn send ([^ZMQ$Socket socket ^bytes message flags] (.send socket message flags)) ([^ZMQ$Socket socket ^bytes message] (send socket message ZMQ/NOBLOCK))) (defn recv-more? [^ZMQ$Socket socket] (.hasReceiveMore socket)) (defn recv ([^ZMQ$Socket socket flags] (.recv socket flags)) ([^ZMQ$Socket socket] (recv socket 0))) ================================================ FILE: storm-core/src/dev/resources/tester_bolt.py ================================================ # -*- coding: utf-8 -*- # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information # regarding copyright ownership. The ASF licenses this file # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at # # http:# www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. # This Python file uses the following encoding: utf-8 import storm from random import random class TesterBolt(storm.Bolt): def initialize(self, conf, context): storm.emit(['bolt initializing']) def process(self, tup): word = tup.values[0]; if (random() < 0.75): storm.emit([word + 'lalala'], anchors=[tup]) storm.ack(tup) else: storm.log(word + ' randomly skipped!') TesterBolt().run() ================================================ FILE: storm-core/src/dev/resources/tester_bolt.rb ================================================ # -*- coding: utf-8 -*- # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information # regarding copyright ownership. The ASF licenses this file # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at # # http:# www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. require File.expand_path("storm", File.dirname(__FILE__)) class TesterBolt < Storm::Bolt def prepare(conf, context) emit ['bolt initializing'] end def process(tuple) word = tuple.values[0] if (rand < 0.75) emit [word + "lalala"], :anchor => tuple ack tuple else log(word + ' randomly skipped!') end end end TesterBolt.new.run ================================================ FILE: storm-core/src/dev/resources/tester_spout.py ================================================ # -*- coding: utf-8 -*- # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information # regarding copyright ownership. The ASF licenses this file # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at # # http:# www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. # This Python file uses the following encoding: utf-8 from storm import Spout, emit, log from random import choice from time import sleep from uuid import uuid4 words = [u"nathan", u"mike", u"jackson", u"golda", u"bertels人"] class TesterSpout(Spout): def initialize(self, conf, context): emit(['spout initializing']) self.pending = {} def nextTuple(self): sleep(1.0/2) word = choice(words) id = str(uuid4()) self.pending[id] = word emit([word], id=id) def ack(self, id): del self.pending[id] def fail(self, id): log("emitting " + self.pending[id] + " on fail") emit([self.pending[id]], id=id) TesterSpout().run() ================================================ FILE: storm-core/src/dev/resources/tester_spout.rb ================================================ # -*- coding: utf-8 -*- # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information # regarding copyright ownership. The ASF licenses this file # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at # # http:# www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. require File.expand_path("storm", File.dirname(__FILE__)) $words = ["nathan", "mike", "jackson", "golda", "bertels人"] def random_word $words[rand($words.length)] end class TesterSpout < Storm::Spout attr_accessor :uid, :pending def open(conf, context) emit ['spout initializing'] self.pending = {} self.uid = 0 end def nextTuple sleep 0.5 word = random_word id = self.uid += 1 self.pending[id] = word emit [word], :id => id end def ack(id) self.pending.delete(id) end def fail(id) word = self.pending[id] log "emitting " + word + " on fail" emit [word], :id => id end end TesterSpout.new.run ================================================ FILE: storm-core/src/genthrift.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. rm -rf gen-javabean gen-py py rm -rf jvm/backtype/storm/generated thrift7 --gen java:beans,hashcode,nocamel --gen py:utf8strings storm.thrift mv gen-javabean/backtype/storm/generated jvm/backtype/storm/generated mv gen-py py rm -rf gen-javabean ================================================ FILE: storm-core/src/jvm/backtype/storm/Config.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm; import backtype.storm.ConfigValidation; import backtype.storm.serialization.IKryoDecorator; import backtype.storm.serialization.IKryoFactory; import com.esotericsoftware.kryo.Serializer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; /** * Topology configs are specified as a plain old map. This class provides a * convenient way to create a topology config map by providing setter methods for * all the configs that can be set. It also makes it easier to do things like add * serializations. * *

This class also provides constants for all the configurations possible on * a Storm cluster and Storm topology. Each constant is paired with a schema * that defines the validity criterion of the corresponding field. Default * values for these configs can be found in defaults.yaml.

* *

Note that you may put other configurations in any of the configs. Storm * will ignore anything it doesn't recognize, but your topologies are free to make * use of them by reading them in the prepare method of Bolts or the open method of * Spouts.

*/ public class Config extends HashMap { /** * The transporter for communication among Storm tasks */ public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport"; public static final Object STORM_MESSAGING_TRANSPORT_SCHEMA = String.class; /** * Netty based messaging: The buffer size for send/recv buffer */ public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = Number.class; /** * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible */ public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = Number.class; /** * Netty based messaging: The min # of milliseconds that a peer will wait. */ public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = Number.class; /** * Netty based messaging: The max # of milliseconds that a peer will wait. */ public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = Number.class; /** * Netty based messaging: The # of worker threads for the server. */ public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads"; public static final Object STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS_SCHEMA = Number.class; /** * Netty based messaging: The # of worker threads for the client. */ public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; public static final Object STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS_SCHEMA = Number.class; /** * A list of hosts of ZooKeeper servers used to manage the cluster. */ public static final String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers"; public static final Object STORM_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator; /** * The port Storm will use to connect to each of the ZooKeeper servers. */ public static final String STORM_ZOOKEEPER_PORT = "storm.zookeeper.port"; public static final Object STORM_ZOOKEEPER_PORT_SCHEMA = Number.class; /** * A directory on the local filesystem used by Storm for any local * filesystem usage it needs. The directory must exist and the Storm daemons must * have permission to read/write from this location. */ public static final String STORM_LOCAL_DIR = "storm.local.dir"; public static final Object STORM_LOCAL_DIR_SCHEMA = String.class; /** * A global task scheduler used to assign topologies's tasks to supervisors' wokers. * * If this is not set, a default system scheduler will be used. */ public static final String STORM_SCHEDULER = "storm.scheduler"; public static final Object STORM_SCHEDULER_SCHEMA = String.class; /** * The mode this Storm cluster is running in. Either "distributed" or "local". */ public static final String STORM_CLUSTER_MODE = "storm.cluster.mode"; public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class; /** * The hostname the supervisors/workers should report to nimbus. If unset, Storm will * get the hostname to report by calling InetAddress.getLocalHost().getCanonicalHostName(). * * You should set this config when you dont have a DNS which supervisors/workers * can utilize to find each other based on hostname got from calls to * InetAddress.getLocalHost().getCanonicalHostName(). */ public static final String STORM_LOCAL_HOSTNAME = "storm.local.hostname"; public static final Object STORM_LOCAL_HOSTNAME_SCHEMA = String.class; /** * The transport plug-in for Thrift client/server communication */ public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport"; public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; /** * The serializer class for ListDelegate (tuple payload). * The default serializer will be ListDelegateSerializer */ public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer"; public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class; /** * Whether or not to use ZeroMQ for messaging in local mode. If this is set * to false, then Storm will use a pure-Java messaging system. The purpose * of this flag is to make it easy to run Storm in local mode by eliminating * the need for native dependencies, which can be difficult to install. * * Defaults to false. */ public static final String STORM_LOCAL_MODE_ZMQ = "storm.local.mode.zmq"; public static final Object STORM_LOCAL_MODE_ZMQ_SCHEMA = Boolean.class; /** * The root location at which Storm stores data in ZooKeeper. */ public static final String STORM_ZOOKEEPER_ROOT = "storm.zookeeper.root"; public static final Object STORM_ZOOKEEPER_ROOT_SCHEMA = String.class; /** * The session timeout for clients to ZooKeeper. */ public static final String STORM_ZOOKEEPER_SESSION_TIMEOUT = "storm.zookeeper.session.timeout"; public static final Object STORM_ZOOKEEPER_SESSION_TIMEOUT_SCHEMA = Number.class; /** * The connection timeout for clients to ZooKeeper. */ public static final String STORM_ZOOKEEPER_CONNECTION_TIMEOUT = "storm.zookeeper.connection.timeout"; public static final Object STORM_ZOOKEEPER_CONNECTION_TIMEOUT_SCHEMA = Number.class; /** * The number of times to retry a Zookeeper operation. */ public static final String STORM_ZOOKEEPER_RETRY_TIMES="storm.zookeeper.retry.times"; public static final Object STORM_ZOOKEEPER_RETRY_TIMES_SCHEMA = Number.class; /** * The interval between retries of a Zookeeper operation. */ public static final String STORM_ZOOKEEPER_RETRY_INTERVAL="storm.zookeeper.retry.interval"; public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_SCHEMA = Number.class; /** * The ceiling of the interval between retries of a Zookeeper operation. */ public static final String STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING="storm.zookeeper.retry.intervalceiling.millis"; public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING_SCHEMA = Number.class; /** * The Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication. */ public static final String STORM_ZOOKEEPER_AUTH_SCHEME="storm.zookeeper.auth.scheme"; public static final Object STORM_ZOOKEEPER_AUTH_SCHEME_SCHEMA = String.class; /** * A string representing the payload for Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication. */ public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD="storm.zookeeper.auth.payload"; public static final Object STORM_ZOOKEEPER_AUTH_PAYLOAD_SCHEMA = String.class; /** * The id assigned to a running topology. The id is the storm name with a unique nonce appended. */ public static final String STORM_ID = "storm.id"; public static final Object STORM_ID_SCHEMA = String.class; /** * The host that the master server is running on. */ public static final String NIMBUS_HOST = "nimbus.host"; public static final Object NIMBUS_HOST_SCHEMA = String.class; /** * Which port the Thrift interface of Nimbus should run on. Clients should * connect to this port to upload jars and submit topologies. */ public static final String NIMBUS_THRIFT_PORT = "nimbus.thrift.port"; public static final Object NIMBUS_THRIFT_PORT_SCHEMA = Number.class; /** * This parameter is used by the storm-deploy project to configure the * jvm options for the nimbus daemon. */ public static final String NIMBUS_CHILDOPTS = "nimbus.childopts"; public static final Object NIMBUS_CHILDOPTS_SCHEMA = String.class; /** * How long without heartbeating a task can go before nimbus will consider the * task dead and reassign it to another location. */ public static final String NIMBUS_TASK_TIMEOUT_SECS = "nimbus.task.timeout.secs"; public static final Object NIMBUS_TASK_TIMEOUT_SECS_SCHEMA = Number.class; /** * How often nimbus should wake up to check heartbeats and do reassignments. Note * that if a machine ever goes down Nimbus will immediately wake up and take action. * This parameter is for checking for failures when there's no explicit event like that * occuring. */ public static final String NIMBUS_MONITOR_FREQ_SECS = "nimbus.monitor.freq.secs"; public static final Object NIMBUS_MONITOR_FREQ_SECS_SCHEMA = Number.class; /** * How often nimbus should wake the cleanup thread to clean the inbox. * @see NIMBUS_INBOX_JAR_EXPIRATION_SECS */ public static final String NIMBUS_CLEANUP_INBOX_FREQ_SECS = "nimbus.cleanup.inbox.freq.secs"; public static final Object NIMBUS_CLEANUP_INBOX_FREQ_SECS_SCHEMA = Number.class; /** * The length of time a jar file lives in the inbox before being deleted by the cleanup thread. * * Probably keep this value greater than or equal to NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS. * Note that the time it takes to delete an inbox jar file is going to be somewhat more than * NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS (depending on how often NIMBUS_CLEANUP_FREQ_SECS * is set to). * @see NIMBUS_CLEANUP_FREQ_SECS */ public static final String NIMBUS_INBOX_JAR_EXPIRATION_SECS = "nimbus.inbox.jar.expiration.secs"; public static final Object NIMBUS_INBOX_JAR_EXPIRATION_SECS_SCHEMA = Number.class; /** * How long before a supervisor can go without heartbeating before nimbus considers it dead * and stops assigning new work to it. */ public static final String NIMBUS_SUPERVISOR_TIMEOUT_SECS = "nimbus.supervisor.timeout.secs"; public static final Object NIMBUS_SUPERVISOR_TIMEOUT_SECS_SCHEMA = Number.class; /** * A special timeout used when a task is initially launched. During launch, this is the timeout * used until the first heartbeat, overriding nimbus.task.timeout.secs. * *

A separate timeout exists for launch because there can be quite a bit of overhead * to launching new JVM's and configuring them.

*/ public static final String NIMBUS_TASK_LAUNCH_SECS = "nimbus.task.launch.secs"; public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = Number.class; /** * Whether or not nimbus should reassign tasks if it detects that a task goes down. * Defaults to true, and it's not recommended to change this value. */ public static final String NIMBUS_REASSIGN = "nimbus.reassign"; public static final Object NIMBUS_REASSIGN_SCHEMA = Boolean.class; /** * During upload/download with the master, how long an upload or download connection is idle * before nimbus considers it dead and drops the connection. */ public static final String NIMBUS_FILE_COPY_EXPIRATION_SECS = "nimbus.file.copy.expiration.secs"; public static final Object NIMBUS_FILE_COPY_EXPIRATION_SECS_SCHEMA = Number.class; /** * A custom class that implements ITopologyValidator that is run whenever a * topology is submitted. Can be used to provide business-specific logic for * whether topologies are allowed to run or not. */ public static final String NIMBUS_TOPOLOGY_VALIDATOR = "nimbus.topology.validator"; public static final Object NIMBUS_TOPOLOGY_VALIDATOR_SCHEMA = String.class; /** * Class name for authorization plugin for Nimbus */ public static final String NIMBUS_AUTHORIZER = "nimbus.authorizer"; public static final Object NIMBUS_AUTHORIZER_SCHEMA = String.class; /** * Storm UI binds to this port. */ public static final String UI_PORT = "ui.port"; public static final Object UI_PORT_SCHEMA = Number.class; /** * HTTP UI port for log viewer */ public static final String LOGVIEWER_PORT = "logviewer.port"; public static final Object LOGVIEWER_PORT_SCHEMA = Number.class; /** * Childopts for log viewer java process. */ public static final String LOGVIEWER_CHILDOPTS = "logviewer.childopts"; public static final Object LOGVIEWER_CHILDOPTS_SCHEMA = String.class; /** * Appender name used by log viewer to determine log directory. */ public static final String LOGVIEWER_APPENDER_NAME = "logviewer.appender.name"; public static final Object LOGVIEWER_APPENDER_NAME_SCHEMA = String.class; /** * Childopts for Storm UI Java process. */ public static final String UI_CHILDOPTS = "ui.childopts"; public static final Object UI_CHILDOPTS_SCHEMA = String.class; /** * List of DRPC servers so that the DRPCSpout knows who to talk to. */ public static final String DRPC_SERVERS = "drpc.servers"; public static final Object DRPC_SERVERS_SCHEMA = ConfigValidation.StringsValidator; /** * This port is used by Storm DRPC for receiving DPRC requests from clients. */ public static final String DRPC_PORT = "drpc.port"; public static final Object DRPC_PORT_SCHEMA = Number.class; /** * DRPC thrift server worker threads */ public static final String DRPC_WORKER_THREADS = "drpc.worker.threads"; public static final Object DRPC_WORKER_THREADS_SCHEMA = Number.class; /** * DRPC thrift server queue size */ public static final String DRPC_QUEUE_SIZE = "drpc.queue.size"; public static final Object DRPC_QUEUE_SIZE_SCHEMA = Number.class; /** * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back. */ public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port"; public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = Number.class; /** * The timeout on DRPC requests within the DRPC server. Defaults to 10 minutes. Note that requests can also * timeout based on the socket timeout on the DRPC client, and separately based on the topology message * timeout for the topology implementing the DRPC function. */ public static final String DRPC_REQUEST_TIMEOUT_SECS = "drpc.request.timeout.secs"; public static final Object DRPC_REQUEST_TIMEOUT_SECS_SCHEMA = Number.class; /** * Childopts for Storm DRPC Java process. */ public static final String DRPC_CHILDOPTS = "drpc.childopts"; public static final Object DRPC_CHILDOPTS_SCHEMA = String.class; /** * the metadata configed on the supervisor */ public static final String SUPERVISOR_SCHEDULER_META = "supervisor.scheduler.meta"; public static final Object SUPERVISOR_SCHEDULER_META_SCHEMA = Map.class; /** * A list of ports that can run workers on this supervisor. Each worker uses one port, and * the supervisor will only run one worker per port. Use this configuration to tune * how many workers run on each machine. */ public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports"; public static final Object SUPERVISOR_SLOTS_PORTS_SCHEMA = ConfigValidation.NumbersValidator; /** * This parameter is used by the storm-deploy project to configure the * jvm options for the supervisor daemon. */ public static final String SUPERVISOR_CHILDOPTS = "supervisor.childopts"; public static final Object SUPERVISOR_CHILDOPTS_SCHEMA = String.class; /** * How long a worker can go without heartbeating before the supervisor tries to * restart the worker process. */ public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs"; public static final Object SUPERVISOR_WORKER_TIMEOUT_SECS_SCHEMA = Number.class; /** * How long a worker can go without heartbeating during the initial launch before * the supervisor tries to restart the worker process. This value override * supervisor.worker.timeout.secs during launch because there is additional * overhead to starting and configuring the JVM on launch. */ public static final String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs"; public static final Object SUPERVISOR_WORKER_START_TIMEOUT_SECS_SCHEMA = Number.class; /** * Whether or not the supervisor should launch workers assigned to it. Defaults * to true -- and you should probably never change this value. This configuration * is used in the Storm unit tests. */ public static final String SUPERVISOR_ENABLE = "supervisor.enable"; public static final Object SUPERVISOR_ENABLE_SCHEMA = Boolean.class; /** * how often the supervisor sends a heartbeat to the master. */ public static final String SUPERVISOR_HEARTBEAT_FREQUENCY_SECS = "supervisor.heartbeat.frequency.secs"; public static final Object SUPERVISOR_HEARTBEAT_FREQUENCY_SECS_SCHEMA = Number.class; /** * How often the supervisor checks the worker heartbeats to see if any of them * need to be restarted. */ public static final String SUPERVISOR_MONITOR_FREQUENCY_SECS = "supervisor.monitor.frequency.secs"; public static final Object SUPERVISOR_MONITOR_FREQUENCY_SECS_SCHEMA = Number.class; /** * The jvm opts provided to workers launched by this supervisor. All "%ID%" substrings are replaced * with an identifier for this worker. */ public static final String WORKER_CHILDOPTS = "worker.childopts"; public static final Object WORKER_CHILDOPTS_SCHEMA = String.class; /** * How often this worker should heartbeat to the supervisor. */ public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs"; public static final Object WORKER_HEARTBEAT_FREQUENCY_SECS_SCHEMA = Number.class; /** * How often a task should heartbeat its status to the master. */ public static final String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs"; public static final Object TASK_HEARTBEAT_FREQUENCY_SECS_SCHEMA = Number.class; /** * How often a task should sync its connections with other tasks (if a task is * reassigned, the other tasks sending messages to it need to refresh their connections). * In general though, when a reassignment happens other tasks will be notified * almost immediately. This configuration is here just in case that notification doesn't * come through. */ public static final String TASK_REFRESH_POLL_SECS = "task.refresh.poll.secs"; public static final Object TASK_REFRESH_POLL_SECS_SCHEMA = Number.class; /** * True if Storm should timeout messages or not. Defaults to true. This is meant to be used * in unit tests to prevent tuples from being accidentally timed out during the test. */ public static final String TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS = "topology.enable.message.timeouts"; public static final Object TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS_SCHEMA = Boolean.class; /** * When set to true, Storm will log every message that's emitted. */ public static final String TOPOLOGY_DEBUG = "topology.debug"; public static final Object TOPOLOGY_DEBUG_SCHEMA = Boolean.class; /** * Whether or not the master should optimize topologies by running multiple * tasks in a single thread where appropriate. */ public static final String TOPOLOGY_OPTIMIZE = "topology.optimize"; public static final Object TOPOLOGY_OPTIMIZE_SCHEMA = Boolean.class; /** * How many processes should be spawned around the cluster to execute this * topology. Each process will execute some number of tasks as threads within * them. This parameter should be used in conjunction with the parallelism hints * on each component in the topology to tune the performance of a topology. */ public static final String TOPOLOGY_WORKERS = "topology.workers"; public static final Object TOPOLOGY_WORKERS_SCHEMA = Number.class; /** * How many instances to create for a spout/bolt. A task runs on a thread with zero or more * other tasks for the same spout/bolt. The number of tasks for a spout/bolt is always * the same throughout the lifetime of a topology, but the number of executors (threads) for * a spout/bolt can change over time. This allows a topology to scale to more or less resources * without redeploying the topology or violating the constraints of Storm (such as a fields grouping * guaranteeing that the same value goes to the same task). */ public static final String TOPOLOGY_TASKS = "topology.tasks"; public static final Object TOPOLOGY_TASKS_SCHEMA = Number.class; /** * How many executors to spawn for ackers. * *

If this is set to 0, then Storm will immediately ack tuples as soon * as they come off the spout, effectively disabling reliability.

*/ public static final String TOPOLOGY_ACKER_EXECUTORS = "topology.acker.executors"; public static final Object TOPOLOGY_ACKER_EXECUTORS_SCHEMA = Number.class; /** * The maximum amount of time given to the topology to fully process a message * emitted by a spout. If the message is not acked within this time frame, Storm * will fail the message on the spout. Some spouts implementations will then replay * the message at a later time. */ public static final String TOPOLOGY_MESSAGE_TIMEOUT_SECS = "topology.message.timeout.secs"; public static final Object TOPOLOGY_MESSAGE_TIMEOUT_SECS_SCHEMA = Number.class; /** * A list of serialization registrations for Kryo ( http://code.google.com/p/kryo/ ), * the underlying serialization framework for Storm. A serialization can either * be the name of a class (in which case Kryo will automatically create a serializer for the class * that saves all the object's fields), or an implementation of com.esotericsoftware.kryo.Serializer. * * See Kryo's documentation for more information about writing custom serializers. */ public static final String TOPOLOGY_KRYO_REGISTER = "topology.kryo.register"; public static final Object TOPOLOGY_KRYO_REGISTER_SCHEMA = ConfigValidation.StringsValidator; /** * A list of classes that customize storm's kryo instance during start-up. * Each listed class name must implement IKryoDecorator. During start-up the * listed class is instantiated with 0 arguments, then its 'decorate' method * is called with storm's kryo instance as the only argument. */ public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators"; public static final Object TOPOLOGY_KRYO_DECORATORS_SCHEMA = ConfigValidation.StringsValidator; /** * Class that specifies how to create a Kryo instance for serialization. Storm will then apply * topology.kryo.register and topology.kryo.decorators on top of this. The default implementation * implements topology.fall.back.on.java.serialization and turns references off. */ public static final String TOPOLOGY_KRYO_FACTORY = "topology.kryo.factory"; public static final Object TOPOLOGY_KRYO_FACTORY_SCHEMA = String.class; /** * Whether or not Storm should skip the loading of kryo registrations for which it * does not know the class or have the serializer implementation. Otherwise, the task will * fail to load and will throw an error at runtime. The use case of this is if you want to * declare your serializations on the storm.yaml files on the cluster rather than every single * time you submit a topology. Different applications may use different serializations and so * a single application may not have the code for the other serializers used by other apps. * By setting this config to true, Storm will ignore that it doesn't have those other serializations * rather than throw an error. */ public static final String TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS= "topology.skip.missing.kryo.registrations"; public static final Object TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS_SCHEMA = Boolean.class; /* * A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format). * Each listed class will be routed all the metrics data generated by the storm metrics API. * Each listed class maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable. */ public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register"; public static final Object TOPOLOGY_METRICS_CONSUMER_REGISTER_SCHEMA = ConfigValidation.MapsValidator; /** * The maximum parallelism allowed for a component in this topology. This configuration is * typically used in testing to limit the number of threads spawned in local mode. */ public static final String TOPOLOGY_MAX_TASK_PARALLELISM="topology.max.task.parallelism"; public static final Object TOPOLOGY_MAX_TASK_PARALLELISM_SCHEMA = Number.class; /** * The maximum number of tuples that can be pending on a spout task at any given time. * This config applies to individual tasks, not to spouts or topologies as a whole. * * A pending tuple is one that has been emitted from a spout but has not been acked or failed yet. * Note that this config parameter has no effect for unreliable spouts that don't tag * their tuples with a message id. */ public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = Number.class; /** * A class that implements a strategy for what to do when a spout needs to wait. Waiting is * triggered in one of two conditions: * * 1. nextTuple emits no tuples * 2. The spout has hit maxSpoutPending and can't emit any more tuples */ public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy"; public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class; /** * The amount of milliseconds the SleepEmptyEmitStrategy should sleep for. */ public static final String TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS="topology.sleep.spout.wait.strategy.time.ms"; public static final Object TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS_SCHEMA = Number.class; /** * The maximum amount of time a component gives a source of state to synchronize before it requests * synchronization again. */ public static final String TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS="topology.state.synchronization.timeout.secs"; public static final Object TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS_SCHEMA = Number.class; /** * The percentage of tuples to sample to produce stats for a task. */ public static final String TOPOLOGY_STATS_SAMPLE_RATE="topology.stats.sample.rate"; public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA = Number.class; /** * The time period that builtin metrics data in bucketed into. */ public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs"; public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = Number.class; /** * Whether or not to use Java serialization in a topology. */ public static final String TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION="topology.fall.back.on.java.serialization"; public static final Object TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION_SCHEMA = Boolean.class; /** * Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS. */ public static final String TOPOLOGY_WORKER_CHILDOPTS="topology.worker.childopts"; public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = String.class; /** * This config is available for TransactionalSpouts, and contains the id ( a String) for * the transactional topology. This id is used to store the state of the transactional * topology in Zookeeper. */ public static final String TOPOLOGY_TRANSACTIONAL_ID="topology.transactional.id"; public static final Object TOPOLOGY_TRANSACTIONAL_ID_SCHEMA = String.class; /** * A list of task hooks that are automatically added to every spout and bolt in the topology. An example * of when you'd do this is to add a hook that integrates with your internal * monitoring system. These hooks are instantiated using the zero-arg constructor. */ public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks"; public static final Object TOPOLOGY_AUTO_TASK_HOOKS_SCHEMA = ConfigValidation.StringsValidator; /** * The size of the Disruptor receive queue for each executor. Must be a power of 2. */ public static final String TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE="topology.executor.receive.buffer.size"; public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; /** * The maximum number of messages to batch from the thread receiving off the network to the * executor queues. Must be a power of 2. */ public static final String TOPOLOGY_RECEIVER_BUFFER_SIZE="topology.receiver.buffer.size"; public static final Object TOPOLOGY_RECEIVER_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; /** * The size of the Disruptor send queue for each executor. Must be a power of 2. */ public static final String TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE="topology.executor.send.buffer.size"; public static final Object TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; /** * The size of the Disruptor transfer queue for each worker. */ public static final String TOPOLOGY_TRANSFER_BUFFER_SIZE="topology.transfer.buffer.size"; public static final Object TOPOLOGY_TRANSFER_BUFFER_SIZE_SCHEMA = Number.class; /** * How often a tick tuple from the "__system" component and "__tick" stream should be sent * to tasks. Meant to be used as a component-specific configuration. */ public static final String TOPOLOGY_TICK_TUPLE_FREQ_SECS="topology.tick.tuple.freq.secs"; public static final Object TOPOLOGY_TICK_TUPLE_FREQ_SECS_SCHEMA = Number.class; /** * Configure the wait strategy used for internal queuing. Can be used to tradeoff latency * vs. throughput */ public static final String TOPOLOGY_DISRUPTOR_WAIT_STRATEGY="topology.disruptor.wait.strategy"; public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class; /** * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed * via the TopologyContext. */ public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size"; public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = Number.class; /** * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, * an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be * reported to Zookeeper per task for every 10 second interval of time. */ public static final String TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS="topology.error.throttle.interval.secs"; public static final Object TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS_SCHEMA = Number.class; /** * See doc for TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS */ public static final String TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL="topology.max.error.report.per.interval"; public static final Object TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL_SCHEMA = Number.class; /** * How often a batch can be emitted in a Trident topology. */ public static final String TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS="topology.trident.batch.emit.interval.millis"; public static final Object TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS_SCHEMA = Number.class; /** * Name of the topology. This config is automatically set by Storm when the topology is submitted. */ public static final String TOPOLOGY_NAME="topology.name"; public static final Object TOPOLOGY_NAME_SCHEMA = String.class; /** * Max pending tuples in one ShellBolt */ public static final String TOPOLOGY_SHELLBOLT_MAX_PENDING="topology.shellbolt.max.pending"; public static final Object TOPOLOGY_SHELLBOLT_MAX_PENDING_SCHEMA = Number.class; /** * The root directory in ZooKeeper for metadata about TransactionalSpouts. */ public static final String TRANSACTIONAL_ZOOKEEPER_ROOT="transactional.zookeeper.root"; public static final Object TRANSACTIONAL_ZOOKEEPER_ROOT_SCHEMA = String.class; /** * The list of zookeeper servers in which to keep the transactional state. If null (which is default), * will use storm.zookeeper.servers */ public static final String TRANSACTIONAL_ZOOKEEPER_SERVERS="transactional.zookeeper.servers"; public static final Object TRANSACTIONAL_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator; /** * The port to use to connect to the transactional zookeeper servers. If null (which is default), * will use storm.zookeeper.port */ public static final String TRANSACTIONAL_ZOOKEEPER_PORT="transactional.zookeeper.port"; public static final Object TRANSACTIONAL_ZOOKEEPER_PORT_SCHEMA = Number.class; /** * The number of threads that should be used by the zeromq context in each worker process. */ public static final String ZMQ_THREADS = "zmq.threads"; public static final Object ZMQ_THREADS_SCHEMA = Number.class; /** * How long a connection should retry sending messages to a target host when * the connection is closed. This is an advanced configuration and can almost * certainly be ignored. */ public static final String ZMQ_LINGER_MILLIS = "zmq.linger.millis"; public static final Object ZMQ_LINGER_MILLIS_SCHEMA = Number.class; /** * The high water for the ZeroMQ push sockets used for networking. Use this config to prevent buffer explosion * on the networking layer. */ public static final String ZMQ_HWM = "zmq.hwm"; public static final Object ZMQ_HWM_SCHEMA = Number.class; /** * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers) * for the java.library.path value. java.library.path tells the JVM where * to look for native libraries. It is necessary to set this config correctly since * Storm uses the ZeroMQ and JZMQ native libs. */ public static final String JAVA_LIBRARY_PATH = "java.library.path"; public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class; /** * The path to use as the zookeeper dir when running a zookeeper server via * "storm dev-zookeeper". This zookeeper instance is only intended for development; * it is not a production grade zookeeper setup. */ public static final String DEV_ZOOKEEPER_PATH = "dev.zookeeper.path"; public static final Object DEV_ZOOKEEPER_PATH_SCHEMA = String.class; /** * A map from topology name to the number of machines that should be dedicated for that topology. Set storm.scheduler * to backtype.storm.scheduler.IsolationScheduler to make use of the isolation scheduler. */ public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines"; public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = Map.class; public static void setDebug(Map conf, boolean isOn) { conf.put(Config.TOPOLOGY_DEBUG, isOn); } public void setDebug(boolean isOn) { setDebug(this, isOn); } @Deprecated public void setOptimize(boolean isOn) { put(Config.TOPOLOGY_OPTIMIZE, isOn); } public static void setNumWorkers(Map conf, int workers) { conf.put(Config.TOPOLOGY_WORKERS, workers); } public void setNumWorkers(int workers) { setNumWorkers(this, workers); } public static void setNumAckers(Map conf, int numExecutors) { conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, numExecutors); } public void setNumAckers(int numExecutors) { setNumAckers(this, numExecutors); } public static void setMessageTimeoutSecs(Map conf, int secs) { conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs); } public void setMessageTimeoutSecs(int secs) { setMessageTimeoutSecs(this, secs); } public static void registerSerialization(Map conf, Class klass) { getRegisteredSerializations(conf).add(klass.getName()); } public void registerSerialization(Class klass) { registerSerialization(this, klass); } public static void registerSerialization(Map conf, Class klass, Class serializerClass) { Map register = new HashMap(); register.put(klass.getName(), serializerClass.getName()); getRegisteredSerializations(conf).add(register); } public void registerSerialization(Class klass, Class serializerClass) { registerSerialization(this, klass, serializerClass); } public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) { HashMap m = new HashMap(); m.put("class", klass.getCanonicalName()); m.put("parallelism.hint", parallelismHint); m.put("argument", argument); List l = (List)this.get(TOPOLOGY_METRICS_CONSUMER_REGISTER); if(l == null) { l = new ArrayList(); } l.add(m); this.put(TOPOLOGY_METRICS_CONSUMER_REGISTER, l); } public void registerMetricsConsumer(Class klass, long parallelismHint) { registerMetricsConsumer(klass, null, parallelismHint); } public void registerMetricsConsumer(Class klass) { registerMetricsConsumer(klass, null, 1L); } public static void registerDecorator(Map conf, Class klass) { getRegisteredDecorators(conf).add(klass.getName()); } public void registerDecorator(Class klass) { registerDecorator(this, klass); } public static void setKryoFactory(Map conf, Class klass) { conf.put(Config.TOPOLOGY_KRYO_FACTORY, klass.getName()); } public void setKryoFactory(Class klass) { setKryoFactory(this, klass); } public static void setSkipMissingKryoRegistrations(Map conf, boolean skip) { conf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, skip); } public void setSkipMissingKryoRegistrations(boolean skip) { setSkipMissingKryoRegistrations(this, skip); } public static void setMaxTaskParallelism(Map conf, int max) { conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max); } public void setMaxTaskParallelism(int max) { setMaxTaskParallelism(this, max); } public static void setMaxSpoutPending(Map conf, int max) { conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max); } public void setMaxSpoutPending(int max) { setMaxSpoutPending(this, max); } public static void setStatsSampleRate(Map conf, double rate) { conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, rate); } public void setStatsSampleRate(double rate) { setStatsSampleRate(this, rate); } public static void setFallBackOnJavaSerialization(Map conf, boolean fallback) { conf.put(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION, fallback); } public void setFallBackOnJavaSerialization(boolean fallback) { setFallBackOnJavaSerialization(this, fallback); } private static List getRegisteredSerializations(Map conf) { List ret; if(!conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) { ret = new ArrayList(); } else { ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_REGISTER)); } conf.put(Config.TOPOLOGY_KRYO_REGISTER, ret); return ret; } private static List getRegisteredDecorators(Map conf) { List ret; if(!conf.containsKey(Config.TOPOLOGY_KRYO_DECORATORS)) { ret = new ArrayList(); } else { ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS)); } conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret); return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/ConfigValidation.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm; import java.util.Map; /** * Provides functionality for validating configuration fields. */ public class ConfigValidation { /** * Declares methods for validating configuration values. */ public static interface FieldValidator { /** * Validates the given field. * @param name the name of the field. * @param field The field to be validated. * @throws IllegalArgumentException if the field fails validation. */ public void validateField(String name, Object field) throws IllegalArgumentException; } /** * Returns a new FieldValidator for a List of the given Class. * @param cls the Class of elements composing the list * @return a FieldValidator for a list of the given class */ static FieldValidator FieldListValidatorFactory(final Class cls) { return new FieldValidator() { @Override public void validateField(String name, Object field) throws IllegalArgumentException { if (field == null) { // A null value is acceptable. return; } if (field instanceof Iterable) { for (Object e : (Iterable)field) { if (! cls.isInstance(e)) { throw new IllegalArgumentException( "Each element of the list " + name + " must be a " + cls.getName() + "."); } } return; } throw new IllegalArgumentException( "Field " + name + " must be an Iterable of " + cls.getName()); } }; } /** * Validates a list of Numbers. */ public static Object NumbersValidator = FieldListValidatorFactory(Number.class); /** * Validates is a list of Strings. */ public static Object StringsValidator = FieldListValidatorFactory(String.class); /** * Validates is a list of Maps. */ public static Object MapsValidator = FieldListValidatorFactory(Map.class); /** * Validates a power of 2. */ public static Object PowerOf2Validator = new FieldValidator() { @Override public void validateField(String name, Object o) throws IllegalArgumentException { if (o == null) { // A null value is acceptable. return; } final long i; if (o instanceof Number && (i = ((Number)o).longValue()) == ((Number)o).doubleValue()) { // Test whether the integer is a power of 2. if (i > 0 && (i & (i-1)) == 0) { return; } } throw new IllegalArgumentException("Field " + name + " must be a power of 2."); } }; } ================================================ FILE: storm-core/src/jvm/backtype/storm/Constants.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm; import backtype.storm.coordination.CoordinatedBolt; import clojure.lang.RT; public class Constants { public static final String COORDINATED_STREAM_ID = CoordinatedBolt.class.getName() + "/coord-stream"; public static final long SYSTEM_TASK_ID = -1; public static final Object SYSTEM_EXECUTOR_ID = RT.readString("[-1 -1]"); public static final String SYSTEM_COMPONENT_ID = "__system"; public static final String SYSTEM_TICK_STREAM_ID = "__tick"; public static final String METRICS_COMPONENT_ID_PREFIX = "__metrics"; public static final String METRICS_STREAM_ID = "__metrics"; public static final String METRICS_TICK_STREAM_ID = "__metrics_tick"; } ================================================ FILE: storm-core/src/jvm/backtype/storm/ILocalCluster.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm; import backtype.storm.generated.AlreadyAliveException; import backtype.storm.generated.ClusterSummary; import backtype.storm.generated.InvalidTopologyException; import backtype.storm.generated.KillOptions; import backtype.storm.generated.SubmitOptions; import backtype.storm.generated.NotAliveException; import backtype.storm.generated.RebalanceOptions; import backtype.storm.generated.StormTopology; import backtype.storm.generated.TopologyInfo; import java.util.Map; public interface ILocalCluster { void submitTopology(String topologyName, Map conf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException; void submitTopologyWithOpts(String topologyName, Map conf, StormTopology topology, SubmitOptions submitOpts) throws AlreadyAliveException, InvalidTopologyException; void killTopology(String topologyName) throws NotAliveException; void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException; void activate(String topologyName) throws NotAliveException; void deactivate(String topologyName) throws NotAliveException; void rebalance(String name, RebalanceOptions options) throws NotAliveException; void shutdown(); String getTopologyConf(String id); StormTopology getTopology(String id); ClusterSummary getClusterInfo(); TopologyInfo getTopologyInfo(String id); Map getState(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/ILocalDRPC.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm; import backtype.storm.daemon.Shutdownable; import backtype.storm.generated.DistributedRPC; import backtype.storm.generated.DistributedRPCInvocations; public interface ILocalDRPC extends DistributedRPC.Iface, DistributedRPCInvocations.Iface, Shutdownable { public String getServiceId(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/StormSubmitter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm; import backtype.storm.generated.*; import backtype.storm.utils.BufferFileInputStream; import backtype.storm.utils.NimbusClient; import backtype.storm.utils.Utils; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.thrift7.TException; import org.json.simple.JSONValue; /** * Use this class to submit topologies to run on the Storm cluster. You should run your program * with the "storm jar" command from the command-line, and then use this class to * submit your topologies. */ public class StormSubmitter { public static Logger LOG = LoggerFactory.getLogger(StormSubmitter.class); private static Nimbus.Iface localNimbus = null; public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) { StormSubmitter.localNimbus = localNimbusHandler; } /** * Submits a topology to run on the cluster. A topology runs forever or until * explicitly killed. * * * @param name the name of the storm. * @param stormConf the topology-specific configuration. See {@link Config}. * @param topology the processing to execute. * @throws AlreadyAliveException if a topology with this name is already running * @throws InvalidTopologyException if an invalid topology was submitted */ public static void submitTopology(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException { submitTopology(name, stormConf, topology, null); } /** * Submits a topology to run on the cluster. A topology runs forever or until * explicitly killed. * * * @param name the name of the storm. * @param stormConf the topology-specific configuration. See {@link Config}. * @param topology the processing to execute. * @param options to manipulate the starting of the topology * @throws AlreadyAliveException if a topology with this name is already running * @throws InvalidTopologyException if an invalid topology was submitted */ public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts) throws AlreadyAliveException, InvalidTopologyException { if(!Utils.isValidConf(stormConf)) { throw new IllegalArgumentException("Storm conf is not valid. Must be json-serializable"); } stormConf = new HashMap(stormConf); stormConf.putAll(Utils.readCommandLineOpts()); Map conf = Utils.readStormConfig(); conf.putAll(stormConf); try { String serConf = JSONValue.toJSONString(stormConf); if(localNimbus!=null) { LOG.info("Submitting topology " + name + " in local mode"); localNimbus.submitTopology(name, null, serConf, topology); } else { NimbusClient client = NimbusClient.getConfiguredClient(conf); if(topologyNameExists(conf, name)) { throw new RuntimeException("Topology with name `" + name + "` already exists on cluster"); } submitJar(conf); try { LOG.info("Submitting topology " + name + " in distributed mode with conf " + serConf); if(opts!=null) { client.getClient().submitTopologyWithOpts(name, submittedJar, serConf, topology, opts); } else { // this is for backwards compatibility client.getClient().submitTopology(name, submittedJar, serConf, topology); } } catch(InvalidTopologyException e) { LOG.warn("Topology submission exception", e); throw e; } catch(AlreadyAliveException e) { LOG.warn("Topology already alive exception", e); throw e; } finally { client.close(); } } LOG.info("Finished submitting topology: " + name); } catch(TException e) { throw new RuntimeException(e); } } private static boolean topologyNameExists(Map conf, String name) { NimbusClient client = NimbusClient.getConfiguredClient(conf); try { ClusterSummary summary = client.getClient().getClusterInfo(); for(TopologySummary s : summary.get_topologies()) { if(s.get_name().equals(name)) { return true; } } return false; } catch(Exception e) { throw new RuntimeException(e); } finally { client.close(); } } private static String submittedJar = null; private static void submitJar(Map conf) { if(submittedJar==null) { LOG.info("Jar not uploaded to master yet. Submitting jar..."); String localJar = System.getProperty("storm.jar"); submittedJar = submitJar(conf, localJar); } else { LOG.info("Jar already uploaded to master. Not submitting jar."); } } public static String submitJar(Map conf, String localJar) { if(localJar==null) { throw new RuntimeException("Must submit topologies using the 'storm' client script so that StormSubmitter knows which jar to upload."); } NimbusClient client = NimbusClient.getConfiguredClient(conf); try { String uploadLocation = client.getClient().beginFileUpload(); LOG.info("Uploading topology jar " + localJar + " to assigned location: " + uploadLocation); BufferFileInputStream is = new BufferFileInputStream(localJar); while(true) { byte[] toSubmit = is.read(); if(toSubmit.length==0) break; client.getClient().uploadChunk(uploadLocation, ByteBuffer.wrap(toSubmit)); } client.getClient().finishFileUpload(uploadLocation); LOG.info("Successfully uploaded topology jar to assigned location: " + uploadLocation); return uploadLocation; } catch(Exception e) { throw new RuntimeException(e); } finally { client.close(); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/clojure/ClojureBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.clojure; import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; import backtype.storm.generated.StreamInfo; import backtype.storm.task.IBolt; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.utils.Utils; import clojure.lang.IFn; import clojure.lang.PersistentArrayMap; import clojure.lang.Keyword; import clojure.lang.Symbol; import clojure.lang.RT; import java.util.ArrayList; import java.util.List; import java.util.Map; public class ClojureBolt implements IRichBolt, FinishedCallback { Map _fields; List _fnSpec; List _confSpec; List _params; IBolt _bolt; public ClojureBolt(List fnSpec, List confSpec, List params, Map fields) { _fnSpec = fnSpec; _confSpec = confSpec; _params = params; _fields = fields; } @Override public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) { IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1)); try { IFn preparer = (IFn) hof.applyTo(RT.seq(_params)); final Map collectorMap = new PersistentArrayMap( new Object[] { Keyword.intern(Symbol.create("output-collector")), collector, Keyword.intern(Symbol.create("context")), context}); List args = new ArrayList() {{ add(stormConf); add(context); add(collectorMap); }}; _bolt = (IBolt) preparer.applyTo(RT.seq(args)); //this is kind of unnecessary for clojure try { _bolt.prepare(stormConf, context, collector); } catch(AbstractMethodError ame) { } } catch (Exception e) { throw new RuntimeException(e); } } @Override public void execute(Tuple input) { _bolt.execute(input); } @Override public void cleanup() { try { _bolt.cleanup(); } catch(AbstractMethodError ame) { } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { for(String stream: _fields.keySet()) { StreamInfo info = _fields.get(stream); declarer.declareStream(stream, info.is_direct(), new Fields(info.get_output_fields())); } } @Override public void finishedId(Object id) { if(_bolt instanceof FinishedCallback) { ((FinishedCallback) _bolt).finishedId(id); } } @Override public Map getComponentConfiguration() { IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1)); try { return (Map) hof.applyTo(RT.seq(_params)); } catch (Exception e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/clojure/ClojureSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.clojure; import backtype.storm.generated.StreamInfo; import backtype.storm.spout.ISpout; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichSpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import clojure.lang.IFn; import clojure.lang.PersistentArrayMap; import clojure.lang.Keyword; import clojure.lang.Symbol; import clojure.lang.RT; import java.util.ArrayList; import java.util.List; import java.util.Map; public class ClojureSpout implements IRichSpout { Map _fields; List _fnSpec; List _confSpec; List _params; ISpout _spout; public ClojureSpout(List fnSpec, List confSpec, List params, Map fields) { _fnSpec = fnSpec; _confSpec = confSpec; _params = params; _fields = fields; } @Override public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) { IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1)); try { IFn preparer = (IFn) hof.applyTo(RT.seq(_params)); final Map collectorMap = new PersistentArrayMap( new Object[] { Keyword.intern(Symbol.create("output-collector")), collector, Keyword.intern(Symbol.create("context")), context}); List args = new ArrayList() {{ add(conf); add(context); add(collectorMap); }}; _spout = (ISpout) preparer.applyTo(RT.seq(args)); //this is kind of unnecessary for clojure try { _spout.open(conf, context, collector); } catch(AbstractMethodError ame) { } } catch (Exception e) { throw new RuntimeException(e); } } @Override public void close() { try { _spout.close(); } catch(AbstractMethodError ame) { } } @Override public void nextTuple() { try { _spout.nextTuple(); } catch(AbstractMethodError ame) { } } @Override public void ack(Object msgId) { try { _spout.ack(msgId); } catch(AbstractMethodError ame) { } } @Override public void fail(Object msgId) { try { _spout.fail(msgId); } catch(AbstractMethodError ame) { } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { for(String stream: _fields.keySet()) { StreamInfo info = _fields.get(stream); declarer.declareStream(stream, info.is_direct(), new Fields(info.get_output_fields())); } } @Override public Map getComponentConfiguration() { IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1)); try { return (Map) hof.applyTo(RT.seq(_params)); } catch (Exception e) { throw new RuntimeException(e); } } @Override public void activate() { try { _spout.activate(); } catch(AbstractMethodError ame) { } } @Override public void deactivate() { try { _spout.deactivate(); } catch(AbstractMethodError ame) { } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.clojure; import backtype.storm.generated.StreamInfo; import backtype.storm.task.ShellBolt; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import java.util.Map; public class RichShellBolt extends ShellBolt implements IRichBolt { private Map _outputs; public RichShellBolt(String[] command, Map outputs) { super(command); _outputs = outputs; } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { for(String stream: _outputs.keySet()) { StreamInfo def = _outputs.get(stream); if(def.is_direct()) { declarer.declareStream(stream, true, new Fields(def.get_output_fields())); } else { declarer.declareStream(stream, new Fields(def.get_output_fields())); } } } @Override public Map getComponentConfiguration() { return null; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.clojure; import backtype.storm.generated.StreamInfo; import backtype.storm.spout.ShellSpout; import backtype.storm.topology.IRichSpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import java.util.Map; public class RichShellSpout extends ShellSpout implements IRichSpout { private Map _outputs; public RichShellSpout(String[] command, Map outputs) { super(command); _outputs = outputs; } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { for(String stream: _outputs.keySet()) { StreamInfo def = _outputs.get(stream); if(def.is_direct()) { declarer.declareStream(stream, true, new Fields(def.get_output_fields())); } else { declarer.declareStream(stream, new Fields(def.get_output_fields())); } } } @Override public Map getComponentConfiguration() { return null; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.coordination; import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; import backtype.storm.coordination.CoordinatedBolt.TimeoutCallback; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.FailedException; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import backtype.storm.utils.Utils; import java.util.HashMap; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCallback { public static Logger LOG = LoggerFactory.getLogger(BatchBoltExecutor.class); byte[] _boltSer; Map _openTransactions; Map _conf; TopologyContext _context; BatchOutputCollectorImpl _collector; public BatchBoltExecutor(IBatchBolt bolt) { _boltSer = Utils.serialize(bolt); } @Override public void prepare(Map conf, TopologyContext context, OutputCollector collector) { _conf = conf; _context = context; _collector = new BatchOutputCollectorImpl(collector); _openTransactions = new HashMap(); } @Override public void execute(Tuple input) { Object id = input.getValue(0); IBatchBolt bolt = getBatchBolt(id); try { bolt.execute(input); _collector.ack(input); } catch(FailedException e) { LOG.error("Failed to process tuple in batch", e); _collector.fail(input); } } @Override public void cleanup() { } @Override public void finishedId(Object id) { IBatchBolt bolt = getBatchBolt(id); _openTransactions.remove(id); bolt.finishBatch(); } @Override public void timeoutId(Object attempt) { _openTransactions.remove(attempt); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { newTransactionalBolt().declareOutputFields(declarer); } @Override public Map getComponentConfiguration() { return newTransactionalBolt().getComponentConfiguration(); } private IBatchBolt getBatchBolt(Object id) { IBatchBolt bolt = _openTransactions.get(id); if(bolt==null) { bolt = newTransactionalBolt(); bolt.prepare(_conf, _context, _collector, id); _openTransactions.put(id, bolt); } return bolt; } private IBatchBolt newTransactionalBolt() { return (IBatchBolt) Utils.deserialize(_boltSer); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/coordination/BatchOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.coordination; import backtype.storm.utils.Utils; import java.util.List; public abstract class BatchOutputCollector { /** * Emits a tuple to the default output stream. */ public List emit(List tuple) { return emit(Utils.DEFAULT_STREAM_ID, tuple); } public abstract List emit(String streamId, List tuple); /** * Emits a tuple to the specified task on the default output stream. This output * stream must have been declared as a direct stream, and the specified task must * use a direct grouping on this stream to receive the message. */ public void emitDirect(int taskId, List tuple) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); } public abstract void emitDirect(int taskId, String streamId, List tuple); public abstract void reportError(Throwable error); } ================================================ FILE: storm-core/src/jvm/backtype/storm/coordination/BatchOutputCollectorImpl.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.coordination; import backtype.storm.task.OutputCollector; import backtype.storm.tuple.Tuple; import java.util.List; public class BatchOutputCollectorImpl extends BatchOutputCollector { OutputCollector _collector; public BatchOutputCollectorImpl(OutputCollector collector) { _collector = collector; } @Override public List emit(String streamId, List tuple) { return _collector.emit(streamId, tuple); } @Override public void emitDirect(int taskId, String streamId, List tuple) { _collector.emitDirect(taskId, streamId, tuple); } @Override public void reportError(Throwable error) { _collector.reportError(error); } public void ack(Tuple tup) { _collector.ack(tup); } public void fail(Tuple tup) { _collector.fail(tup); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/coordination/BatchSubtopologyBuilder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.coordination; import backtype.storm.Constants; import backtype.storm.coordination.CoordinatedBolt.SourceArgs; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.topology.BaseConfigurationDeclarer; import backtype.storm.topology.BasicBoltExecutor; import backtype.storm.topology.BoltDeclarer; import backtype.storm.topology.IBasicBolt; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.InputDeclarer; import backtype.storm.topology.TopologyBuilder; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; public class BatchSubtopologyBuilder { Map _bolts = new HashMap(); Component _masterBolt; String _masterId; public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt, Number boltParallelism) { Integer p = boltParallelism == null ? null : boltParallelism.intValue(); _masterBolt = new Component(new BasicBoltExecutor(masterBolt), p); _masterId = masterBoltId; } public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt) { this(masterBoltId, masterBolt, null); } public BoltDeclarer getMasterDeclarer() { return new BoltDeclarerImpl(_masterBolt); } public BoltDeclarer setBolt(String id, IBatchBolt bolt) { return setBolt(id, bolt, null); } public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number parallelism) { return setBolt(id, new BatchBoltExecutor(bolt), parallelism); } public BoltDeclarer setBolt(String id, IBasicBolt bolt) { return setBolt(id, bolt, null); } public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism) { return setBolt(id, new BasicBoltExecutor(bolt), parallelism); } private BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism) { Integer p = null; if(parallelism!=null) p = parallelism.intValue(); Component component = new Component(bolt, p); _bolts.put(id, component); return new BoltDeclarerImpl(component); } public void extendTopology(TopologyBuilder builder) { BoltDeclarer declarer = builder.setBolt(_masterId, new CoordinatedBolt(_masterBolt.bolt), _masterBolt.parallelism); for(InputDeclaration decl: _masterBolt.declarations) { decl.declare(declarer); } for(Map conf: _masterBolt.componentConfs) { declarer.addConfigurations(conf); } for(String id: _bolts.keySet()) { Component component = _bolts.get(id); Map coordinatedArgs = new HashMap(); for(String c: componentBoltSubscriptions(component)) { SourceArgs source; if(c.equals(_masterId)) { source = SourceArgs.single(); } else { source = SourceArgs.all(); } coordinatedArgs.put(c, source); } BoltDeclarer input = builder.setBolt(id, new CoordinatedBolt(component.bolt, coordinatedArgs, null), component.parallelism); for(Map conf: component.componentConfs) { input.addConfigurations(conf); } for(String c: componentBoltSubscriptions(component)) { input.directGrouping(c, Constants.COORDINATED_STREAM_ID); } for(InputDeclaration d: component.declarations) { d.declare(input); } } } private Set componentBoltSubscriptions(Component component) { Set ret = new HashSet(); for(InputDeclaration d: component.declarations) { ret.add(d.getComponent()); } return ret; } private static class Component { public IRichBolt bolt; public Integer parallelism; public List declarations = new ArrayList(); public List componentConfs = new ArrayList(); public Component(IRichBolt bolt, Integer parallelism) { this.bolt = bolt; this.parallelism = parallelism; } } private static interface InputDeclaration { void declare(InputDeclarer declarer); String getComponent(); } private class BoltDeclarerImpl extends BaseConfigurationDeclarer implements BoltDeclarer { Component _component; public BoltDeclarerImpl(Component component) { _component = component; } @Override public BoltDeclarer fieldsGrouping(final String component, final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.fieldsGrouping(component, fields); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer fieldsGrouping(final String component, final String streamId, final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.fieldsGrouping(component, streamId, fields); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer globalGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.globalGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer globalGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.globalGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer shuffleGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.shuffleGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer shuffleGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.shuffleGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer localOrShuffleGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.localOrShuffleGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer localOrShuffleGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.localOrShuffleGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer noneGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.noneGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer noneGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.noneGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer allGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.allGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer allGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.allGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer directGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.directGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer directGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.directGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer customGrouping(final String component, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.customGrouping(component, grouping); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer customGrouping(final String component, final String streamId, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.customGrouping(component, streamId, grouping); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer grouping(final GlobalStreamId stream, final Grouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.grouping(stream, grouping); } @Override public String getComponent() { return stream.get_componentId(); } }); return this; } private void addDeclaration(InputDeclaration declaration) { _component.declarations.add(declaration); } @Override public BoltDeclarer addConfigurations(Map conf) { _component.componentConfs.add(conf); return this; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/coordination/CoordinatedBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.coordination; import backtype.storm.topology.FailedException; import java.util.Map.Entry; import backtype.storm.tuple.Values; import backtype.storm.generated.GlobalStreamId; import java.util.Collection; import backtype.storm.Constants; import backtype.storm.generated.Grouping; import backtype.storm.task.IOutputCollector; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.utils.TimeCacheMap; import backtype.storm.utils.Utils; import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static backtype.storm.utils.Utils.get; /** * Coordination requires the request ids to be globally unique for awhile. This is so it doesn't get confused * in the case of retries. */ public class CoordinatedBolt implements IRichBolt { public static Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class); public static interface FinishedCallback { void finishedId(Object id); } public static interface TimeoutCallback { void timeoutId(Object id); } public static class SourceArgs implements Serializable { public boolean singleCount; protected SourceArgs(boolean singleCount) { this.singleCount = singleCount; } public static SourceArgs single() { return new SourceArgs(true); } public static SourceArgs all() { return new SourceArgs(false); } @Override public String toString() { return ""; } } public class CoordinatedOutputCollector implements IOutputCollector { IOutputCollector _delegate; public CoordinatedOutputCollector(IOutputCollector delegate) { _delegate = delegate; } public List emit(String stream, Collection anchors, List tuple) { List tasks = _delegate.emit(stream, anchors, tuple); updateTaskCounts(tuple.get(0), tasks); return tasks; } public void emitDirect(int task, String stream, Collection anchors, List tuple) { updateTaskCounts(tuple.get(0), Arrays.asList(task)); _delegate.emitDirect(task, stream, anchors, tuple); } public void ack(Tuple tuple) { Object id = tuple.getValue(0); synchronized(_tracked) { TrackingInfo track = _tracked.get(id); if (track != null) track.receivedTuples++; } boolean failed = checkFinishId(tuple, TupleType.REGULAR); if(failed) { _delegate.fail(tuple); } else { _delegate.ack(tuple); } } public void fail(Tuple tuple) { Object id = tuple.getValue(0); synchronized(_tracked) { TrackingInfo track = _tracked.get(id); if (track != null) track.failed = true; } checkFinishId(tuple, TupleType.REGULAR); _delegate.fail(tuple); } public void reportError(Throwable error) { _delegate.reportError(error); } private void updateTaskCounts(Object id, List tasks) { synchronized(_tracked) { TrackingInfo track = _tracked.get(id); if (track != null) { Map taskEmittedTuples = track.taskEmittedTuples; for(Integer task: tasks) { int newCount = get(taskEmittedTuples, task, 0) + 1; taskEmittedTuples.put(task, newCount); } } } } } private Map _sourceArgs; private IdStreamSpec _idStreamSpec; private IRichBolt _delegate; private Integer _numSourceReports; private List _countOutTasks = new ArrayList();; private OutputCollector _collector; private TimeCacheMap _tracked; public static class TrackingInfo { int reportCount = 0; int expectedTupleCount = 0; int receivedTuples = 0; boolean failed = false; Map taskEmittedTuples = new HashMap(); boolean receivedId = false; boolean finished = false; List ackTuples = new ArrayList(); @Override public String toString() { return "reportCount: " + reportCount + "\n" + "expectedTupleCount: " + expectedTupleCount + "\n" + "receivedTuples: " + receivedTuples + "\n" + "failed: " + failed + "\n" + taskEmittedTuples.toString(); } } public static class IdStreamSpec implements Serializable { GlobalStreamId _id; public GlobalStreamId getGlobalStreamId() { return _id; } public static IdStreamSpec makeDetectSpec(String component, String stream) { return new IdStreamSpec(component, stream); } protected IdStreamSpec(String component, String stream) { _id = new GlobalStreamId(component, stream); } } public CoordinatedBolt(IRichBolt delegate) { this(delegate, null, null); } public CoordinatedBolt(IRichBolt delegate, String sourceComponent, SourceArgs sourceArgs, IdStreamSpec idStreamSpec) { this(delegate, singleSourceArgs(sourceComponent, sourceArgs), idStreamSpec); } public CoordinatedBolt(IRichBolt delegate, Map sourceArgs, IdStreamSpec idStreamSpec) { _sourceArgs = sourceArgs; if(_sourceArgs==null) _sourceArgs = new HashMap(); _delegate = delegate; _idStreamSpec = idStreamSpec; } public void prepare(Map config, TopologyContext context, OutputCollector collector) { TimeCacheMap.ExpiredCallback callback = null; if(_delegate instanceof TimeoutCallback) { callback = new TimeoutItems(); } _tracked = new TimeCacheMap(context.maxTopologyMessageTimeout(), callback); _collector = collector; _delegate.prepare(config, context, new OutputCollector(new CoordinatedOutputCollector(collector))); for(String component: Utils.get(context.getThisTargets(), Constants.COORDINATED_STREAM_ID, new HashMap()) .keySet()) { for(Integer task: context.getComponentTasks(component)) { _countOutTasks.add(task); } } if(!_sourceArgs.isEmpty()) { _numSourceReports = 0; for(Entry entry: _sourceArgs.entrySet()) { if(entry.getValue().singleCount) { _numSourceReports+=1; } else { _numSourceReports+=context.getComponentTasks(entry.getKey()).size(); } } } } private boolean checkFinishId(Tuple tup, TupleType type) { Object id = tup.getValue(0); boolean failed = false; synchronized(_tracked) { TrackingInfo track = _tracked.get(id); try { if(track!=null) { boolean delayed = false; if(_idStreamSpec==null && type == TupleType.COORD || _idStreamSpec!=null && type==TupleType.ID) { track.ackTuples.add(tup); delayed = true; } if(track.failed) { failed = true; for(Tuple t: track.ackTuples) { _collector.fail(t); } _tracked.remove(id); } else if(track.receivedId && (_sourceArgs.isEmpty() || track.reportCount==_numSourceReports && track.expectedTupleCount == track.receivedTuples)){ if(_delegate instanceof FinishedCallback) { ((FinishedCallback)_delegate).finishedId(id); } if(!(_sourceArgs.isEmpty() || type!=TupleType.REGULAR)) { throw new IllegalStateException("Coordination condition met on a non-coordinating tuple. Should be impossible"); } Iterator outTasks = _countOutTasks.iterator(); while(outTasks.hasNext()) { int task = outTasks.next(); int numTuples = get(track.taskEmittedTuples, task, 0); _collector.emitDirect(task, Constants.COORDINATED_STREAM_ID, tup, new Values(id, numTuples)); } for(Tuple t: track.ackTuples) { _collector.ack(t); } track.finished = true; _tracked.remove(id); } if(!delayed && type!=TupleType.REGULAR) { if(track.failed) { _collector.fail(tup); } else { _collector.ack(tup); } } } else { if(type!=TupleType.REGULAR) _collector.fail(tup); } } catch(FailedException e) { LOG.error("Failed to finish batch", e); for(Tuple t: track.ackTuples) { _collector.fail(t); } _tracked.remove(id); failed = true; } } return failed; } public void execute(Tuple tuple) { Object id = tuple.getValue(0); TrackingInfo track; TupleType type = getTupleType(tuple); synchronized(_tracked) { track = _tracked.get(id); if(track==null) { track = new TrackingInfo(); if(_idStreamSpec==null) track.receivedId = true; _tracked.put(id, track); } } if(type==TupleType.ID) { synchronized(_tracked) { track.receivedId = true; } checkFinishId(tuple, type); } else if(type==TupleType.COORD) { int count = (Integer) tuple.getValue(1); synchronized(_tracked) { track.reportCount++; track.expectedTupleCount+=count; } checkFinishId(tuple, type); } else { synchronized(_tracked) { _delegate.execute(tuple); } } } public void cleanup() { _delegate.cleanup(); _tracked.cleanup(); } public void declareOutputFields(OutputFieldsDeclarer declarer) { _delegate.declareOutputFields(declarer); declarer.declareStream(Constants.COORDINATED_STREAM_ID, true, new Fields("id", "count")); } @Override public Map getComponentConfiguration() { return _delegate.getComponentConfiguration(); } private static Map singleSourceArgs(String sourceComponent, SourceArgs sourceArgs) { Map ret = new HashMap(); ret.put(sourceComponent, sourceArgs); return ret; } private class TimeoutItems implements TimeCacheMap.ExpiredCallback { @Override public void expire(Object id, TrackingInfo val) { synchronized(_tracked) { // the combination of the lock and the finished flag ensure that // an id is never timed out if it has been finished val.failed = true; if(!val.finished) { ((TimeoutCallback) _delegate).timeoutId(id); } } } } private TupleType getTupleType(Tuple tuple) { if(_idStreamSpec!=null && tuple.getSourceGlobalStreamid().equals(_idStreamSpec._id)) { return TupleType.ID; } else if(!_sourceArgs.isEmpty() && tuple.getSourceStreamId().equals(Constants.COORDINATED_STREAM_ID)) { return TupleType.COORD; } else { return TupleType.REGULAR; } } static enum TupleType { REGULAR, ID, COORD } } ================================================ FILE: storm-core/src/jvm/backtype/storm/coordination/IBatchBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.coordination; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IComponent; import backtype.storm.tuple.Tuple; import java.io.Serializable; import java.util.Map; public interface IBatchBolt extends Serializable, IComponent { void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, T id); void execute(Tuple tuple); void finishBatch(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/daemon/Shutdownable.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.daemon; public interface Shutdownable { public void shutdown(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.generated.DRPCRequest; import backtype.storm.generated.DistributedRPCInvocations; import org.apache.thrift7.TException; import org.apache.thrift7.protocol.TBinaryProtocol; import org.apache.thrift7.transport.TFramedTransport; import org.apache.thrift7.transport.TSocket; import org.apache.thrift7.transport.TTransport; public class DRPCInvocationsClient implements DistributedRPCInvocations.Iface { private TTransport conn; private DistributedRPCInvocations.Client client; private String host; private int port; public DRPCInvocationsClient(String host, int port) { try { this.host = host; this.port = port; connect(); } catch(TException e) { throw new RuntimeException(e); } } private void connect() throws TException { conn = new TFramedTransport(new TSocket(host, port)); client = new DistributedRPCInvocations.Client(new TBinaryProtocol(conn)); conn.open(); } public String getHost() { return host; } public int getPort() { return port; } public void result(String id, String result) throws TException { try { if(client==null) connect(); client.result(id, result); } catch(TException e) { client = null; throw e; } } public DRPCRequest fetchRequest(String func) throws TException { try { if(client==null) connect(); return client.fetchRequest(func); } catch(TException e) { client = null; throw e; } } public void failRequest(String id) throws TException { try { if(client==null) connect(); client.failRequest(id); } catch(TException e) { client = null; throw e; } } public void close() { conn.close(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.Config; import backtype.storm.ILocalDRPC; import backtype.storm.generated.DRPCRequest; import backtype.storm.generated.DistributedRPCInvocations; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import backtype.storm.utils.ServiceRegistry; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.thrift7.TException; import org.json.simple.JSONValue; public class DRPCSpout extends BaseRichSpout { public static Logger LOG = LoggerFactory.getLogger(DRPCSpout.class); SpoutOutputCollector _collector; List _clients = new ArrayList(); String _function; String _local_drpc_id = null; private static class DRPCMessageId { String id; int index; public DRPCMessageId(String id, int index) { this.id = id; this.index = index; } } public DRPCSpout(String function) { _function = function; } public DRPCSpout(String function, ILocalDRPC drpc) { _function = function; _local_drpc_id = drpc.getServiceId(); } @Override public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _collector = collector; if(_local_drpc_id==null) { int numTasks = context.getComponentTasks(context.getThisComponentId()).size(); int index = context.getThisTaskIndex(); int port = Utils.getInt(conf.get(Config.DRPC_INVOCATIONS_PORT)); List servers = (List) conf.get(Config.DRPC_SERVERS); if(servers == null || servers.isEmpty()) { throw new RuntimeException("No DRPC servers configured for topology"); } if(numTasks < servers.size()) { for(String s: servers) { _clients.add(new DRPCInvocationsClient(s, port)); } } else { int i = index % servers.size(); _clients.add(new DRPCInvocationsClient(servers.get(i), port)); } } } @Override public void close() { for(DRPCInvocationsClient client: _clients) { client.close(); } } @Override public void nextTuple() { boolean gotRequest = false; if(_local_drpc_id==null) { for(int i=0; i<_clients.size(); i++) { DRPCInvocationsClient client = _clients.get(i); try { DRPCRequest req = client.fetchRequest(_function); if(req.get_request_id().length() > 0) { Map returnInfo = new HashMap(); returnInfo.put("id", req.get_request_id()); returnInfo.put("host", client.getHost()); returnInfo.put("port", client.getPort()); gotRequest = true; _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), new DRPCMessageId(req.get_request_id(), i)); break; } } catch (TException e) { LOG.error("Failed to fetch DRPC result from DRPC server", e); } } } else { DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id); if(drpc!=null) { // can happen during shutdown of drpc while topology is still up try { DRPCRequest req = drpc.fetchRequest(_function); if(req.get_request_id().length() > 0) { Map returnInfo = new HashMap(); returnInfo.put("id", req.get_request_id()); returnInfo.put("host", _local_drpc_id); returnInfo.put("port", 0); gotRequest = true; _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), new DRPCMessageId(req.get_request_id(), 0)); } } catch (TException e) { throw new RuntimeException(e); } } } if(!gotRequest) { Utils.sleep(1); } } @Override public void ack(Object msgId) { } @Override public void fail(Object msgId) { DRPCMessageId did = (DRPCMessageId) msgId; DistributedRPCInvocations.Iface client; if(_local_drpc_id == null) { client = _clients.get(did.index); } else { client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id); } try { client.failRequest(did.id); } catch (TException e) { LOG.error("Failed to fail request", e); } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("args", "return-info")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/JoinResult.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseRichBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class JoinResult extends BaseRichBolt { public static Logger LOG = LoggerFactory.getLogger(JoinResult.class); String returnComponent; Map returns = new HashMap(); Map results = new HashMap(); OutputCollector _collector; public JoinResult(String returnComponent) { this.returnComponent = returnComponent; } public void prepare(Map map, TopologyContext context, OutputCollector collector) { _collector = collector; } public void execute(Tuple tuple) { Object requestId = tuple.getValue(0); if(tuple.getSourceComponent().equals(returnComponent)) { returns.put(requestId, tuple); } else { results.put(requestId, tuple); } if(returns.containsKey(requestId) && results.containsKey(requestId)) { Tuple result = results.remove(requestId); Tuple returner = returns.remove(requestId); LOG.debug(result.getValue(1).toString()); List anchors = new ArrayList(); anchors.add(result); anchors.add(returner); _collector.emit(anchors, new Values(""+result.getValue(1), returner.getValue(1))); _collector.ack(result); _collector.ack(returner); } } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("result", "return-info")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/KeyedFairBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.BasicBoltExecutor; import backtype.storm.topology.IBasicBolt; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import backtype.storm.utils.KeyedRoundRobinQueue; import java.util.HashMap; import java.util.Map; public class KeyedFairBolt implements IRichBolt, FinishedCallback { IRichBolt _delegate; KeyedRoundRobinQueue _rrQueue; Thread _executor; FinishedCallback _callback; public KeyedFairBolt(IRichBolt delegate) { _delegate = delegate; } public KeyedFairBolt(IBasicBolt delegate) { this(new BasicBoltExecutor(delegate)); } public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { if(_delegate instanceof FinishedCallback) { _callback = (FinishedCallback) _delegate; } _delegate.prepare(stormConf, context, collector); _rrQueue = new KeyedRoundRobinQueue(); _executor = new Thread(new Runnable() { public void run() { try { while(true) { _delegate.execute(_rrQueue.take()); } } catch (InterruptedException e) { } } }); _executor.setDaemon(true); _executor.start(); } public void execute(Tuple input) { Object key = input.getValue(0); _rrQueue.add(key, input); } public void cleanup() { _executor.interrupt(); _delegate.cleanup(); } public void declareOutputFields(OutputFieldsDeclarer declarer) { _delegate.declareOutputFields(declarer); } public void finishedId(Object id) { if(_callback!=null) { _callback.finishedId(id); } } @Override public Map getComponentConfiguration() { return new HashMap(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/LinearDRPCInputDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.topology.ComponentConfigurationDeclarer; import backtype.storm.tuple.Fields; public interface LinearDRPCInputDeclarer extends ComponentConfigurationDeclarer { public LinearDRPCInputDeclarer fieldsGrouping(Fields fields); public LinearDRPCInputDeclarer fieldsGrouping(String streamId, Fields fields); public LinearDRPCInputDeclarer globalGrouping(); public LinearDRPCInputDeclarer globalGrouping(String streamId); public LinearDRPCInputDeclarer shuffleGrouping(); public LinearDRPCInputDeclarer shuffleGrouping(String streamId); public LinearDRPCInputDeclarer localOrShuffleGrouping(); public LinearDRPCInputDeclarer localOrShuffleGrouping(String streamId); public LinearDRPCInputDeclarer noneGrouping(); public LinearDRPCInputDeclarer noneGrouping(String streamId); public LinearDRPCInputDeclarer allGrouping(); public LinearDRPCInputDeclarer allGrouping(String streamId); public LinearDRPCInputDeclarer directGrouping(); public LinearDRPCInputDeclarer directGrouping(String streamId); public LinearDRPCInputDeclarer customGrouping(CustomStreamGrouping grouping); public LinearDRPCInputDeclarer customGrouping(String streamId, CustomStreamGrouping grouping); } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/LinearDRPCTopologyBuilder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.Constants; import backtype.storm.ILocalDRPC; import backtype.storm.coordination.BatchBoltExecutor; import backtype.storm.coordination.CoordinatedBolt; import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; import backtype.storm.coordination.CoordinatedBolt.IdStreamSpec; import backtype.storm.coordination.CoordinatedBolt.SourceArgs; import backtype.storm.coordination.IBatchBolt; import backtype.storm.generated.StormTopology; import backtype.storm.generated.StreamInfo; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.topology.BaseConfigurationDeclarer; import backtype.storm.topology.BasicBoltExecutor; import backtype.storm.topology.BoltDeclarer; import backtype.storm.topology.IBasicBolt; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.InputDeclarer; import backtype.storm.topology.OutputFieldsGetter; import backtype.storm.topology.TopologyBuilder; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; // Trident subsumes the functionality provided by this class, so it's deprecated @Deprecated public class LinearDRPCTopologyBuilder { String _function; List _components = new ArrayList(); public LinearDRPCTopologyBuilder(String function) { _function = function; } public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt, Number parallelism) { return addBolt(new BatchBoltExecutor(bolt), parallelism); } public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt) { return addBolt(bolt, 1); } @Deprecated public LinearDRPCInputDeclarer addBolt(IRichBolt bolt, Number parallelism) { if(parallelism==null) parallelism = 1; Component component = new Component(bolt, parallelism.intValue()); _components.add(component); return new InputDeclarerImpl(component); } @Deprecated public LinearDRPCInputDeclarer addBolt(IRichBolt bolt) { return addBolt(bolt, null); } public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt, Number parallelism) { return addBolt(new BasicBoltExecutor(bolt), parallelism); } public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt) { return addBolt(bolt, null); } public StormTopology createLocalTopology(ILocalDRPC drpc) { return createTopology(new DRPCSpout(_function, drpc)); } public StormTopology createRemoteTopology() { return createTopology(new DRPCSpout(_function)); } private StormTopology createTopology(DRPCSpout spout) { final String SPOUT_ID = "spout"; final String PREPARE_ID = "prepare-request"; TopologyBuilder builder = new TopologyBuilder(); builder.setSpout(SPOUT_ID, spout); builder.setBolt(PREPARE_ID, new PrepareRequest()) .noneGrouping(SPOUT_ID); int i=0; for(; i<_components.size();i++) { Component component = _components.get(i); Map source = new HashMap(); if (i==1) { source.put(boltId(i-1), SourceArgs.single()); } else if (i>=2) { source.put(boltId(i-1), SourceArgs.all()); } IdStreamSpec idSpec = null; if(i==_components.size()-1 && component.bolt instanceof FinishedCallback) { idSpec = IdStreamSpec.makeDetectSpec(PREPARE_ID, PrepareRequest.ID_STREAM); } BoltDeclarer declarer = builder.setBolt( boltId(i), new CoordinatedBolt(component.bolt, source, idSpec), component.parallelism); for(Map conf: component.componentConfs) { declarer.addConfigurations(conf); } if(idSpec!=null) { declarer.fieldsGrouping(idSpec.getGlobalStreamId().get_componentId(), PrepareRequest.ID_STREAM, new Fields("request")); } if(i==0 && component.declarations.isEmpty()) { declarer.noneGrouping(PREPARE_ID, PrepareRequest.ARGS_STREAM); } else { String prevId; if(i==0) { prevId = PREPARE_ID; } else { prevId = boltId(i-1); } for(InputDeclaration declaration: component.declarations) { declaration.declare(prevId, declarer); } } if(i>0) { declarer.directGrouping(boltId(i-1), Constants.COORDINATED_STREAM_ID); } } IRichBolt lastBolt = _components.get(_components.size()-1).bolt; OutputFieldsGetter getter = new OutputFieldsGetter(); lastBolt.declareOutputFields(getter); Map streams = getter.getFieldsDeclaration(); if(streams.size()!=1) { throw new RuntimeException("Must declare exactly one stream from last bolt in LinearDRPCTopology"); } String outputStream = streams.keySet().iterator().next(); List fields = streams.get(outputStream).get_output_fields(); if(fields.size()!=2) { throw new RuntimeException("Output stream of last component in LinearDRPCTopology must contain exactly two fields. The first should be the request id, and the second should be the result."); } builder.setBolt(boltId(i), new JoinResult(PREPARE_ID)) .fieldsGrouping(boltId(i-1), outputStream, new Fields(fields.get(0))) .fieldsGrouping(PREPARE_ID, PrepareRequest.RETURN_STREAM, new Fields("request")); i++; builder.setBolt(boltId(i), new ReturnResults()) .noneGrouping(boltId(i-1)); return builder.createTopology(); } private static String boltId(int index) { return "bolt" + index; } private static class Component { public IRichBolt bolt; public int parallelism; public List componentConfs; public List declarations = new ArrayList(); public Component(IRichBolt bolt, int parallelism) { this.bolt = bolt; this.parallelism = parallelism; this.componentConfs = new ArrayList(); } } private static interface InputDeclaration { public void declare(String prevComponent, InputDeclarer declarer); } private class InputDeclarerImpl extends BaseConfigurationDeclarer implements LinearDRPCInputDeclarer { Component _component; public InputDeclarerImpl(Component component) { _component = component; } @Override public LinearDRPCInputDeclarer fieldsGrouping(final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.fieldsGrouping(prevComponent, fields); } }); return this; } @Override public LinearDRPCInputDeclarer fieldsGrouping(final String streamId, final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.fieldsGrouping(prevComponent, streamId, fields); } }); return this; } @Override public LinearDRPCInputDeclarer globalGrouping() { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.globalGrouping(prevComponent); } }); return this; } @Override public LinearDRPCInputDeclarer globalGrouping(final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.globalGrouping(prevComponent, streamId); } }); return this; } @Override public LinearDRPCInputDeclarer shuffleGrouping() { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.shuffleGrouping(prevComponent); } }); return this; } @Override public LinearDRPCInputDeclarer shuffleGrouping(final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.shuffleGrouping(prevComponent, streamId); } }); return this; } @Override public LinearDRPCInputDeclarer localOrShuffleGrouping() { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.localOrShuffleGrouping(prevComponent); } }); return this; } @Override public LinearDRPCInputDeclarer localOrShuffleGrouping(final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.localOrShuffleGrouping(prevComponent, streamId); } }); return this; } @Override public LinearDRPCInputDeclarer noneGrouping() { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.noneGrouping(prevComponent); } }); return this; } @Override public LinearDRPCInputDeclarer noneGrouping(final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.noneGrouping(prevComponent, streamId); } }); return this; } @Override public LinearDRPCInputDeclarer allGrouping() { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.allGrouping(prevComponent); } }); return this; } @Override public LinearDRPCInputDeclarer allGrouping(final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.allGrouping(prevComponent, streamId); } }); return this; } @Override public LinearDRPCInputDeclarer directGrouping() { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.directGrouping(prevComponent); } }); return this; } @Override public LinearDRPCInputDeclarer directGrouping(final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.directGrouping(prevComponent, streamId); } }); return this; } @Override public LinearDRPCInputDeclarer customGrouping(final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.customGrouping(prevComponent, grouping); } }); return this; } @Override public LinearDRPCInputDeclarer customGrouping(final String streamId, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(String prevComponent, InputDeclarer declarer) { declarer.customGrouping(prevComponent, streamId, grouping); } }); return this; } private void addDeclaration(InputDeclaration declaration) { _component.declarations.add(declaration); } @Override public LinearDRPCInputDeclarer addConfigurations(Map conf) { _component.componentConfs.add(conf); return this; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/PrepareRequest.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.task.TopologyContext; import backtype.storm.topology.BasicOutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBasicBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import java.util.Map; import java.util.Random; import backtype.storm.utils.Utils; public class PrepareRequest extends BaseBasicBolt { public static final String ARGS_STREAM = Utils.DEFAULT_STREAM_ID; public static final String RETURN_STREAM = "ret"; public static final String ID_STREAM = "id"; Random rand; @Override public void prepare(Map map, TopologyContext context) { rand = new Random(); } @Override public void execute(Tuple tuple, BasicOutputCollector collector) { String args = tuple.getString(0); String returnInfo = tuple.getString(1); long requestId = rand.nextLong(); collector.emit(ARGS_STREAM, new Values(requestId, args)); collector.emit(RETURN_STREAM, new Values(requestId, returnInfo)); collector.emit(ID_STREAM, new Values(requestId)); } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declareStream(ARGS_STREAM, new Fields("request", "args")); declarer.declareStream(RETURN_STREAM, new Fields("request", "return")); declarer.declareStream(ID_STREAM, new Fields("request")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.drpc; import backtype.storm.Config; import backtype.storm.generated.DistributedRPCInvocations; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseRichBolt; import backtype.storm.tuple.Tuple; import backtype.storm.utils.ServiceRegistry; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.thrift7.TException; import org.json.simple.JSONValue; public class ReturnResults extends BaseRichBolt { public static final Logger LOG = LoggerFactory.getLogger(ReturnResults.class); OutputCollector _collector; boolean local; Map _clients = new HashMap(); @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { _collector = collector; local = stormConf.get(Config.STORM_CLUSTER_MODE).equals("local"); } @Override public void execute(Tuple input) { String result = (String) input.getValue(0); String returnInfo = (String) input.getValue(1); if(returnInfo!=null) { Map retMap = (Map) JSONValue.parse(returnInfo); final String host = (String) retMap.get("host"); final int port = Utils.getInt(retMap.get("port")); String id = (String) retMap.get("id"); DistributedRPCInvocations.Iface client; if(local) { client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host); } else { List server = new ArrayList() {{ add(host); add(port); }}; if(!_clients.containsKey(server)) { _clients.put(server, new DRPCInvocationsClient(host, port)); } client = _clients.get(server); } try { client.result(id, result); _collector.ack(input); } catch(TException e) { LOG.error("Failed to return results to DRPC server", e); _collector.fail(input); } } } @Override public void cleanup() { for(DRPCInvocationsClient c: _clients.values()) { c.close(); } } public void declareOutputFields(OutputFieldsDeclarer declarer) { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class AlreadyAliveException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("AlreadyAliveException"); private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); private String msg; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { MSG((short)1, "msg"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // MSG return MSG; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(AlreadyAliveException.class, metaDataMap); } public AlreadyAliveException() { } public AlreadyAliveException( String msg) { this(); this.msg = msg; } /** * Performs a deep copy on other. */ public AlreadyAliveException(AlreadyAliveException other) { if (other.is_set_msg()) { this.msg = other.msg; } } public AlreadyAliveException deepCopy() { return new AlreadyAliveException(this); } @Override public void clear() { this.msg = null; } public String get_msg() { return this.msg; } public void set_msg(String msg) { this.msg = msg; } public void unset_msg() { this.msg = null; } /** Returns true if field msg is set (has been assigned a value) and false otherwise */ public boolean is_set_msg() { return this.msg != null; } public void set_msg_isSet(boolean value) { if (!value) { this.msg = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case MSG: if (value == null) { unset_msg(); } else { set_msg((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case MSG: return get_msg(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case MSG: return is_set_msg(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof AlreadyAliveException) return this.equals((AlreadyAliveException)that); return false; } public boolean equals(AlreadyAliveException that) { if (that == null) return false; boolean this_present_msg = true && this.is_set_msg(); boolean that_present_msg = true && that.is_set_msg(); if (this_present_msg || that_present_msg) { if (!(this_present_msg && that_present_msg)) return false; if (!this.msg.equals(that.msg)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_msg = true && (is_set_msg()); builder.append(present_msg); if (present_msg) builder.append(msg); return builder.toHashCode(); } public int compareTo(AlreadyAliveException other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; AlreadyAliveException typedOther = (AlreadyAliveException)other; lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); if (lastComparison != 0) { return lastComparison; } if (is_set_msg()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // MSG if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.msg = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.msg != null) { oprot.writeFieldBegin(MSG_FIELD_DESC); oprot.writeString(this.msg); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("AlreadyAliveException("); boolean first = true; sb.append("msg:"); if (this.msg == null) { sb.append("null"); } else { sb.append(this.msg); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_msg()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/Bolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class Bolt implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("Bolt"); private static final org.apache.thrift7.protocol.TField BOLT_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("bolt_object", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift7.protocol.TField("common", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private ComponentObject bolt_object; // required private ComponentCommon common; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { BOLT_OBJECT((short)1, "bolt_object"), COMMON((short)2, "common"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // BOLT_OBJECT return BOLT_OBJECT; case 2: // COMMON return COMMON; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.BOLT_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("bolt_object", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentObject.class))); tmpMap.put(_Fields.COMMON, new org.apache.thrift7.meta_data.FieldMetaData("common", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentCommon.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(Bolt.class, metaDataMap); } public Bolt() { } public Bolt( ComponentObject bolt_object, ComponentCommon common) { this(); this.bolt_object = bolt_object; this.common = common; } /** * Performs a deep copy on other. */ public Bolt(Bolt other) { if (other.is_set_bolt_object()) { this.bolt_object = new ComponentObject(other.bolt_object); } if (other.is_set_common()) { this.common = new ComponentCommon(other.common); } } public Bolt deepCopy() { return new Bolt(this); } @Override public void clear() { this.bolt_object = null; this.common = null; } public ComponentObject get_bolt_object() { return this.bolt_object; } public void set_bolt_object(ComponentObject bolt_object) { this.bolt_object = bolt_object; } public void unset_bolt_object() { this.bolt_object = null; } /** Returns true if field bolt_object is set (has been assigned a value) and false otherwise */ public boolean is_set_bolt_object() { return this.bolt_object != null; } public void set_bolt_object_isSet(boolean value) { if (!value) { this.bolt_object = null; } } public ComponentCommon get_common() { return this.common; } public void set_common(ComponentCommon common) { this.common = common; } public void unset_common() { this.common = null; } /** Returns true if field common is set (has been assigned a value) and false otherwise */ public boolean is_set_common() { return this.common != null; } public void set_common_isSet(boolean value) { if (!value) { this.common = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case BOLT_OBJECT: if (value == null) { unset_bolt_object(); } else { set_bolt_object((ComponentObject)value); } break; case COMMON: if (value == null) { unset_common(); } else { set_common((ComponentCommon)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case BOLT_OBJECT: return get_bolt_object(); case COMMON: return get_common(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case BOLT_OBJECT: return is_set_bolt_object(); case COMMON: return is_set_common(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof Bolt) return this.equals((Bolt)that); return false; } public boolean equals(Bolt that) { if (that == null) return false; boolean this_present_bolt_object = true && this.is_set_bolt_object(); boolean that_present_bolt_object = true && that.is_set_bolt_object(); if (this_present_bolt_object || that_present_bolt_object) { if (!(this_present_bolt_object && that_present_bolt_object)) return false; if (!this.bolt_object.equals(that.bolt_object)) return false; } boolean this_present_common = true && this.is_set_common(); boolean that_present_common = true && that.is_set_common(); if (this_present_common || that_present_common) { if (!(this_present_common && that_present_common)) return false; if (!this.common.equals(that.common)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_bolt_object = true && (is_set_bolt_object()); builder.append(present_bolt_object); if (present_bolt_object) builder.append(bolt_object); boolean present_common = true && (is_set_common()); builder.append(present_common); if (present_common) builder.append(common); return builder.toHashCode(); } public int compareTo(Bolt other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; Bolt typedOther = (Bolt)other; lastComparison = Boolean.valueOf(is_set_bolt_object()).compareTo(typedOther.is_set_bolt_object()); if (lastComparison != 0) { return lastComparison; } if (is_set_bolt_object()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.bolt_object, typedOther.bolt_object); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common()); if (lastComparison != 0) { return lastComparison; } if (is_set_common()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.common, typedOther.common); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // BOLT_OBJECT if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.bolt_object = new ComponentObject(); this.bolt_object.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // COMMON if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.common = new ComponentCommon(); this.common.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.bolt_object != null) { oprot.writeFieldBegin(BOLT_OBJECT_FIELD_DESC); this.bolt_object.write(oprot); oprot.writeFieldEnd(); } if (this.common != null) { oprot.writeFieldBegin(COMMON_FIELD_DESC); this.common.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("Bolt("); boolean first = true; sb.append("bolt_object:"); if (this.bolt_object == null) { sb.append("null"); } else { sb.append(this.bolt_object); } first = false; if (!first) sb.append(", "); sb.append("common:"); if (this.common == null) { sb.append("null"); } else { sb.append(this.common); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_bolt_object()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'bolt_object' is unset! Struct:" + toString()); } if (!is_set_common()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/BoltStats.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class BoltStats implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("BoltStats"); private static final org.apache.thrift7.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift7.protocol.TField("acked", org.apache.thrift7.protocol.TType.MAP, (short)1); private static final org.apache.thrift7.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift7.protocol.TField("failed", org.apache.thrift7.protocol.TType.MAP, (short)2); private static final org.apache.thrift7.protocol.TField PROCESS_MS_AVG_FIELD_DESC = new org.apache.thrift7.protocol.TField("process_ms_avg", org.apache.thrift7.protocol.TType.MAP, (short)3); private static final org.apache.thrift7.protocol.TField EXECUTED_FIELD_DESC = new org.apache.thrift7.protocol.TField("executed", org.apache.thrift7.protocol.TType.MAP, (short)4); private static final org.apache.thrift7.protocol.TField EXECUTE_MS_AVG_FIELD_DESC = new org.apache.thrift7.protocol.TField("execute_ms_avg", org.apache.thrift7.protocol.TType.MAP, (short)5); private Map> acked; // required private Map> failed; // required private Map> process_ms_avg; // required private Map> executed; // required private Map> execute_ms_avg; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ACKED((short)1, "acked"), FAILED((short)2, "failed"), PROCESS_MS_AVG((short)3, "process_ms_avg"), EXECUTED((short)4, "executed"), EXECUTE_MS_AVG((short)5, "execute_ms_avg"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ACKED return ACKED; case 2: // FAILED return FAILED; case 3: // PROCESS_MS_AVG return PROCESS_MS_AVG; case 4: // EXECUTED return EXECUTED; case 5: // EXECUTE_MS_AVG return EXECUTE_MS_AVG; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ACKED, new org.apache.thrift7.meta_data.FieldMetaData("acked", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); tmpMap.put(_Fields.FAILED, new org.apache.thrift7.meta_data.FieldMetaData("failed", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); tmpMap.put(_Fields.PROCESS_MS_AVG, new org.apache.thrift7.meta_data.FieldMetaData("process_ms_avg", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); tmpMap.put(_Fields.EXECUTED, new org.apache.thrift7.meta_data.FieldMetaData("executed", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); tmpMap.put(_Fields.EXECUTE_MS_AVG, new org.apache.thrift7.meta_data.FieldMetaData("execute_ms_avg", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(BoltStats.class, metaDataMap); } public BoltStats() { } public BoltStats( Map> acked, Map> failed, Map> process_ms_avg, Map> executed, Map> execute_ms_avg) { this(); this.acked = acked; this.failed = failed; this.process_ms_avg = process_ms_avg; this.executed = executed; this.execute_ms_avg = execute_ms_avg; } /** * Performs a deep copy on other. */ public BoltStats(BoltStats other) { if (other.is_set_acked()) { Map> __this__acked = new HashMap>(); for (Map.Entry> other_element : other.acked.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__acked_copy_key = other_element_key; Map __this__acked_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); Long other_element_value_element_value = other_element_value_element.getValue(); GlobalStreamId __this__acked_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); Long __this__acked_copy_value_copy_value = other_element_value_element_value; __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value); } __this__acked.put(__this__acked_copy_key, __this__acked_copy_value); } this.acked = __this__acked; } if (other.is_set_failed()) { Map> __this__failed = new HashMap>(); for (Map.Entry> other_element : other.failed.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__failed_copy_key = other_element_key; Map __this__failed_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); Long other_element_value_element_value = other_element_value_element.getValue(); GlobalStreamId __this__failed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); Long __this__failed_copy_value_copy_value = other_element_value_element_value; __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value); } __this__failed.put(__this__failed_copy_key, __this__failed_copy_value); } this.failed = __this__failed; } if (other.is_set_process_ms_avg()) { Map> __this__process_ms_avg = new HashMap>(); for (Map.Entry> other_element : other.process_ms_avg.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__process_ms_avg_copy_key = other_element_key; Map __this__process_ms_avg_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); Double other_element_value_element_value = other_element_value_element.getValue(); GlobalStreamId __this__process_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); Double __this__process_ms_avg_copy_value_copy_value = other_element_value_element_value; __this__process_ms_avg_copy_value.put(__this__process_ms_avg_copy_value_copy_key, __this__process_ms_avg_copy_value_copy_value); } __this__process_ms_avg.put(__this__process_ms_avg_copy_key, __this__process_ms_avg_copy_value); } this.process_ms_avg = __this__process_ms_avg; } if (other.is_set_executed()) { Map> __this__executed = new HashMap>(); for (Map.Entry> other_element : other.executed.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__executed_copy_key = other_element_key; Map __this__executed_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); Long other_element_value_element_value = other_element_value_element.getValue(); GlobalStreamId __this__executed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); Long __this__executed_copy_value_copy_value = other_element_value_element_value; __this__executed_copy_value.put(__this__executed_copy_value_copy_key, __this__executed_copy_value_copy_value); } __this__executed.put(__this__executed_copy_key, __this__executed_copy_value); } this.executed = __this__executed; } if (other.is_set_execute_ms_avg()) { Map> __this__execute_ms_avg = new HashMap>(); for (Map.Entry> other_element : other.execute_ms_avg.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__execute_ms_avg_copy_key = other_element_key; Map __this__execute_ms_avg_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); Double other_element_value_element_value = other_element_value_element.getValue(); GlobalStreamId __this__execute_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); Double __this__execute_ms_avg_copy_value_copy_value = other_element_value_element_value; __this__execute_ms_avg_copy_value.put(__this__execute_ms_avg_copy_value_copy_key, __this__execute_ms_avg_copy_value_copy_value); } __this__execute_ms_avg.put(__this__execute_ms_avg_copy_key, __this__execute_ms_avg_copy_value); } this.execute_ms_avg = __this__execute_ms_avg; } } public BoltStats deepCopy() { return new BoltStats(this); } @Override public void clear() { this.acked = null; this.failed = null; this.process_ms_avg = null; this.executed = null; this.execute_ms_avg = null; } public int get_acked_size() { return (this.acked == null) ? 0 : this.acked.size(); } public void put_to_acked(String key, Map val) { if (this.acked == null) { this.acked = new HashMap>(); } this.acked.put(key, val); } public Map> get_acked() { return this.acked; } public void set_acked(Map> acked) { this.acked = acked; } public void unset_acked() { this.acked = null; } /** Returns true if field acked is set (has been assigned a value) and false otherwise */ public boolean is_set_acked() { return this.acked != null; } public void set_acked_isSet(boolean value) { if (!value) { this.acked = null; } } public int get_failed_size() { return (this.failed == null) ? 0 : this.failed.size(); } public void put_to_failed(String key, Map val) { if (this.failed == null) { this.failed = new HashMap>(); } this.failed.put(key, val); } public Map> get_failed() { return this.failed; } public void set_failed(Map> failed) { this.failed = failed; } public void unset_failed() { this.failed = null; } /** Returns true if field failed is set (has been assigned a value) and false otherwise */ public boolean is_set_failed() { return this.failed != null; } public void set_failed_isSet(boolean value) { if (!value) { this.failed = null; } } public int get_process_ms_avg_size() { return (this.process_ms_avg == null) ? 0 : this.process_ms_avg.size(); } public void put_to_process_ms_avg(String key, Map val) { if (this.process_ms_avg == null) { this.process_ms_avg = new HashMap>(); } this.process_ms_avg.put(key, val); } public Map> get_process_ms_avg() { return this.process_ms_avg; } public void set_process_ms_avg(Map> process_ms_avg) { this.process_ms_avg = process_ms_avg; } public void unset_process_ms_avg() { this.process_ms_avg = null; } /** Returns true if field process_ms_avg is set (has been assigned a value) and false otherwise */ public boolean is_set_process_ms_avg() { return this.process_ms_avg != null; } public void set_process_ms_avg_isSet(boolean value) { if (!value) { this.process_ms_avg = null; } } public int get_executed_size() { return (this.executed == null) ? 0 : this.executed.size(); } public void put_to_executed(String key, Map val) { if (this.executed == null) { this.executed = new HashMap>(); } this.executed.put(key, val); } public Map> get_executed() { return this.executed; } public void set_executed(Map> executed) { this.executed = executed; } public void unset_executed() { this.executed = null; } /** Returns true if field executed is set (has been assigned a value) and false otherwise */ public boolean is_set_executed() { return this.executed != null; } public void set_executed_isSet(boolean value) { if (!value) { this.executed = null; } } public int get_execute_ms_avg_size() { return (this.execute_ms_avg == null) ? 0 : this.execute_ms_avg.size(); } public void put_to_execute_ms_avg(String key, Map val) { if (this.execute_ms_avg == null) { this.execute_ms_avg = new HashMap>(); } this.execute_ms_avg.put(key, val); } public Map> get_execute_ms_avg() { return this.execute_ms_avg; } public void set_execute_ms_avg(Map> execute_ms_avg) { this.execute_ms_avg = execute_ms_avg; } public void unset_execute_ms_avg() { this.execute_ms_avg = null; } /** Returns true if field execute_ms_avg is set (has been assigned a value) and false otherwise */ public boolean is_set_execute_ms_avg() { return this.execute_ms_avg != null; } public void set_execute_ms_avg_isSet(boolean value) { if (!value) { this.execute_ms_avg = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ACKED: if (value == null) { unset_acked(); } else { set_acked((Map>)value); } break; case FAILED: if (value == null) { unset_failed(); } else { set_failed((Map>)value); } break; case PROCESS_MS_AVG: if (value == null) { unset_process_ms_avg(); } else { set_process_ms_avg((Map>)value); } break; case EXECUTED: if (value == null) { unset_executed(); } else { set_executed((Map>)value); } break; case EXECUTE_MS_AVG: if (value == null) { unset_execute_ms_avg(); } else { set_execute_ms_avg((Map>)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ACKED: return get_acked(); case FAILED: return get_failed(); case PROCESS_MS_AVG: return get_process_ms_avg(); case EXECUTED: return get_executed(); case EXECUTE_MS_AVG: return get_execute_ms_avg(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ACKED: return is_set_acked(); case FAILED: return is_set_failed(); case PROCESS_MS_AVG: return is_set_process_ms_avg(); case EXECUTED: return is_set_executed(); case EXECUTE_MS_AVG: return is_set_execute_ms_avg(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof BoltStats) return this.equals((BoltStats)that); return false; } public boolean equals(BoltStats that) { if (that == null) return false; boolean this_present_acked = true && this.is_set_acked(); boolean that_present_acked = true && that.is_set_acked(); if (this_present_acked || that_present_acked) { if (!(this_present_acked && that_present_acked)) return false; if (!this.acked.equals(that.acked)) return false; } boolean this_present_failed = true && this.is_set_failed(); boolean that_present_failed = true && that.is_set_failed(); if (this_present_failed || that_present_failed) { if (!(this_present_failed && that_present_failed)) return false; if (!this.failed.equals(that.failed)) return false; } boolean this_present_process_ms_avg = true && this.is_set_process_ms_avg(); boolean that_present_process_ms_avg = true && that.is_set_process_ms_avg(); if (this_present_process_ms_avg || that_present_process_ms_avg) { if (!(this_present_process_ms_avg && that_present_process_ms_avg)) return false; if (!this.process_ms_avg.equals(that.process_ms_avg)) return false; } boolean this_present_executed = true && this.is_set_executed(); boolean that_present_executed = true && that.is_set_executed(); if (this_present_executed || that_present_executed) { if (!(this_present_executed && that_present_executed)) return false; if (!this.executed.equals(that.executed)) return false; } boolean this_present_execute_ms_avg = true && this.is_set_execute_ms_avg(); boolean that_present_execute_ms_avg = true && that.is_set_execute_ms_avg(); if (this_present_execute_ms_avg || that_present_execute_ms_avg) { if (!(this_present_execute_ms_avg && that_present_execute_ms_avg)) return false; if (!this.execute_ms_avg.equals(that.execute_ms_avg)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_acked = true && (is_set_acked()); builder.append(present_acked); if (present_acked) builder.append(acked); boolean present_failed = true && (is_set_failed()); builder.append(present_failed); if (present_failed) builder.append(failed); boolean present_process_ms_avg = true && (is_set_process_ms_avg()); builder.append(present_process_ms_avg); if (present_process_ms_avg) builder.append(process_ms_avg); boolean present_executed = true && (is_set_executed()); builder.append(present_executed); if (present_executed) builder.append(executed); boolean present_execute_ms_avg = true && (is_set_execute_ms_avg()); builder.append(present_execute_ms_avg); if (present_execute_ms_avg) builder.append(execute_ms_avg); return builder.toHashCode(); } public int compareTo(BoltStats other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; BoltStats typedOther = (BoltStats)other; lastComparison = Boolean.valueOf(is_set_acked()).compareTo(typedOther.is_set_acked()); if (lastComparison != 0) { return lastComparison; } if (is_set_acked()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.acked, typedOther.acked); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_failed()).compareTo(typedOther.is_set_failed()); if (lastComparison != 0) { return lastComparison; } if (is_set_failed()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.failed, typedOther.failed); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_process_ms_avg()).compareTo(typedOther.is_set_process_ms_avg()); if (lastComparison != 0) { return lastComparison; } if (is_set_process_ms_avg()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.process_ms_avg, typedOther.process_ms_avg); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_executed()).compareTo(typedOther.is_set_executed()); if (lastComparison != 0) { return lastComparison; } if (is_set_executed()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.executed, typedOther.executed); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_execute_ms_avg()).compareTo(typedOther.is_set_execute_ms_avg()); if (lastComparison != 0) { return lastComparison; } if (is_set_execute_ms_avg()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.execute_ms_avg, typedOther.execute_ms_avg); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ACKED if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map45 = iprot.readMapBegin(); this.acked = new HashMap>(2*_map45.size); for (int _i46 = 0; _i46 < _map45.size; ++_i46) { String _key47; // required Map _val48; // required _key47 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map49 = iprot.readMapBegin(); _val48 = new HashMap(2*_map49.size); for (int _i50 = 0; _i50 < _map49.size; ++_i50) { GlobalStreamId _key51; // required long _val52; // required _key51 = new GlobalStreamId(); _key51.read(iprot); _val52 = iprot.readI64(); _val48.put(_key51, _val52); } iprot.readMapEnd(); } this.acked.put(_key47, _val48); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // FAILED if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map53 = iprot.readMapBegin(); this.failed = new HashMap>(2*_map53.size); for (int _i54 = 0; _i54 < _map53.size; ++_i54) { String _key55; // required Map _val56; // required _key55 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map57 = iprot.readMapBegin(); _val56 = new HashMap(2*_map57.size); for (int _i58 = 0; _i58 < _map57.size; ++_i58) { GlobalStreamId _key59; // required long _val60; // required _key59 = new GlobalStreamId(); _key59.read(iprot); _val60 = iprot.readI64(); _val56.put(_key59, _val60); } iprot.readMapEnd(); } this.failed.put(_key55, _val56); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // PROCESS_MS_AVG if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map61 = iprot.readMapBegin(); this.process_ms_avg = new HashMap>(2*_map61.size); for (int _i62 = 0; _i62 < _map61.size; ++_i62) { String _key63; // required Map _val64; // required _key63 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map65 = iprot.readMapBegin(); _val64 = new HashMap(2*_map65.size); for (int _i66 = 0; _i66 < _map65.size; ++_i66) { GlobalStreamId _key67; // required double _val68; // required _key67 = new GlobalStreamId(); _key67.read(iprot); _val68 = iprot.readDouble(); _val64.put(_key67, _val68); } iprot.readMapEnd(); } this.process_ms_avg.put(_key63, _val64); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // EXECUTED if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map69 = iprot.readMapBegin(); this.executed = new HashMap>(2*_map69.size); for (int _i70 = 0; _i70 < _map69.size; ++_i70) { String _key71; // required Map _val72; // required _key71 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map73 = iprot.readMapBegin(); _val72 = new HashMap(2*_map73.size); for (int _i74 = 0; _i74 < _map73.size; ++_i74) { GlobalStreamId _key75; // required long _val76; // required _key75 = new GlobalStreamId(); _key75.read(iprot); _val76 = iprot.readI64(); _val72.put(_key75, _val76); } iprot.readMapEnd(); } this.executed.put(_key71, _val72); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 5: // EXECUTE_MS_AVG if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map77 = iprot.readMapBegin(); this.execute_ms_avg = new HashMap>(2*_map77.size); for (int _i78 = 0; _i78 < _map77.size; ++_i78) { String _key79; // required Map _val80; // required _key79 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map81 = iprot.readMapBegin(); _val80 = new HashMap(2*_map81.size); for (int _i82 = 0; _i82 < _map81.size; ++_i82) { GlobalStreamId _key83; // required double _val84; // required _key83 = new GlobalStreamId(); _key83.read(iprot); _val84 = iprot.readDouble(); _val80.put(_key83, _val84); } iprot.readMapEnd(); } this.execute_ms_avg.put(_key79, _val80); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.acked != null) { oprot.writeFieldBegin(ACKED_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.acked.size())); for (Map.Entry> _iter85 : this.acked.entrySet()) { oprot.writeString(_iter85.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter85.getValue().size())); for (Map.Entry _iter86 : _iter85.getValue().entrySet()) { _iter86.getKey().write(oprot); oprot.writeI64(_iter86.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.failed != null) { oprot.writeFieldBegin(FAILED_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.failed.size())); for (Map.Entry> _iter87 : this.failed.entrySet()) { oprot.writeString(_iter87.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter87.getValue().size())); for (Map.Entry _iter88 : _iter87.getValue().entrySet()) { _iter88.getKey().write(oprot); oprot.writeI64(_iter88.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.process_ms_avg != null) { oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.process_ms_avg.size())); for (Map.Entry> _iter89 : this.process_ms_avg.entrySet()) { oprot.writeString(_iter89.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.DOUBLE, _iter89.getValue().size())); for (Map.Entry _iter90 : _iter89.getValue().entrySet()) { _iter90.getKey().write(oprot); oprot.writeDouble(_iter90.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.executed != null) { oprot.writeFieldBegin(EXECUTED_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.executed.size())); for (Map.Entry> _iter91 : this.executed.entrySet()) { oprot.writeString(_iter91.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter91.getValue().size())); for (Map.Entry _iter92 : _iter91.getValue().entrySet()) { _iter92.getKey().write(oprot); oprot.writeI64(_iter92.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.execute_ms_avg != null) { oprot.writeFieldBegin(EXECUTE_MS_AVG_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.execute_ms_avg.size())); for (Map.Entry> _iter93 : this.execute_ms_avg.entrySet()) { oprot.writeString(_iter93.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.DOUBLE, _iter93.getValue().size())); for (Map.Entry _iter94 : _iter93.getValue().entrySet()) { _iter94.getKey().write(oprot); oprot.writeDouble(_iter94.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("BoltStats("); boolean first = true; sb.append("acked:"); if (this.acked == null) { sb.append("null"); } else { sb.append(this.acked); } first = false; if (!first) sb.append(", "); sb.append("failed:"); if (this.failed == null) { sb.append("null"); } else { sb.append(this.failed); } first = false; if (!first) sb.append(", "); sb.append("process_ms_avg:"); if (this.process_ms_avg == null) { sb.append("null"); } else { sb.append(this.process_ms_avg); } first = false; if (!first) sb.append(", "); sb.append("executed:"); if (this.executed == null) { sb.append("null"); } else { sb.append(this.executed); } first = false; if (!first) sb.append(", "); sb.append("execute_ms_avg:"); if (this.execute_ms_avg == null) { sb.append("null"); } else { sb.append(this.execute_ms_avg); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_acked()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'acked' is unset! Struct:" + toString()); } if (!is_set_failed()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'failed' is unset! Struct:" + toString()); } if (!is_set_process_ms_avg()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'process_ms_avg' is unset! Struct:" + toString()); } if (!is_set_executed()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'executed' is unset! Struct:" + toString()); } if (!is_set_execute_ms_avg()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'execute_ms_avg' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ClusterSummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ClusterSummary"); private static final org.apache.thrift7.protocol.TField SUPERVISORS_FIELD_DESC = new org.apache.thrift7.protocol.TField("supervisors", org.apache.thrift7.protocol.TType.LIST, (short)1); private static final org.apache.thrift7.protocol.TField NIMBUS_UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("nimbus_uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)2); private static final org.apache.thrift7.protocol.TField TOPOLOGIES_FIELD_DESC = new org.apache.thrift7.protocol.TField("topologies", org.apache.thrift7.protocol.TType.LIST, (short)3); private List supervisors; // required private int nimbus_uptime_secs; // required private List topologies; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUPERVISORS((short)1, "supervisors"), NIMBUS_UPTIME_SECS((short)2, "nimbus_uptime_secs"), TOPOLOGIES((short)3, "topologies"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // SUPERVISORS return SUPERVISORS; case 2: // NIMBUS_UPTIME_SECS return NIMBUS_UPTIME_SECS; case 3: // TOPOLOGIES return TOPOLOGIES; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __NIMBUS_UPTIME_SECS_ISSET_ID = 0; private BitSet __isset_bit_vector = new BitSet(1); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUPERVISORS, new org.apache.thrift7.meta_data.FieldMetaData("supervisors", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SupervisorSummary.class)))); tmpMap.put(_Fields.NIMBUS_UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("nimbus_uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.TOPOLOGIES, new org.apache.thrift7.meta_data.FieldMetaData("topologies", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TopologySummary.class)))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ClusterSummary.class, metaDataMap); } public ClusterSummary() { } public ClusterSummary( List supervisors, int nimbus_uptime_secs, List topologies) { this(); this.supervisors = supervisors; this.nimbus_uptime_secs = nimbus_uptime_secs; set_nimbus_uptime_secs_isSet(true); this.topologies = topologies; } /** * Performs a deep copy on other. */ public ClusterSummary(ClusterSummary other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_supervisors()) { List __this__supervisors = new ArrayList(); for (SupervisorSummary other_element : other.supervisors) { __this__supervisors.add(new SupervisorSummary(other_element)); } this.supervisors = __this__supervisors; } this.nimbus_uptime_secs = other.nimbus_uptime_secs; if (other.is_set_topologies()) { List __this__topologies = new ArrayList(); for (TopologySummary other_element : other.topologies) { __this__topologies.add(new TopologySummary(other_element)); } this.topologies = __this__topologies; } } public ClusterSummary deepCopy() { return new ClusterSummary(this); } @Override public void clear() { this.supervisors = null; set_nimbus_uptime_secs_isSet(false); this.nimbus_uptime_secs = 0; this.topologies = null; } public int get_supervisors_size() { return (this.supervisors == null) ? 0 : this.supervisors.size(); } public java.util.Iterator get_supervisors_iterator() { return (this.supervisors == null) ? null : this.supervisors.iterator(); } public void add_to_supervisors(SupervisorSummary elem) { if (this.supervisors == null) { this.supervisors = new ArrayList(); } this.supervisors.add(elem); } public List get_supervisors() { return this.supervisors; } public void set_supervisors(List supervisors) { this.supervisors = supervisors; } public void unset_supervisors() { this.supervisors = null; } /** Returns true if field supervisors is set (has been assigned a value) and false otherwise */ public boolean is_set_supervisors() { return this.supervisors != null; } public void set_supervisors_isSet(boolean value) { if (!value) { this.supervisors = null; } } public int get_nimbus_uptime_secs() { return this.nimbus_uptime_secs; } public void set_nimbus_uptime_secs(int nimbus_uptime_secs) { this.nimbus_uptime_secs = nimbus_uptime_secs; set_nimbus_uptime_secs_isSet(true); } public void unset_nimbus_uptime_secs() { __isset_bit_vector.clear(__NIMBUS_UPTIME_SECS_ISSET_ID); } /** Returns true if field nimbus_uptime_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_nimbus_uptime_secs() { return __isset_bit_vector.get(__NIMBUS_UPTIME_SECS_ISSET_ID); } public void set_nimbus_uptime_secs_isSet(boolean value) { __isset_bit_vector.set(__NIMBUS_UPTIME_SECS_ISSET_ID, value); } public int get_topologies_size() { return (this.topologies == null) ? 0 : this.topologies.size(); } public java.util.Iterator get_topologies_iterator() { return (this.topologies == null) ? null : this.topologies.iterator(); } public void add_to_topologies(TopologySummary elem) { if (this.topologies == null) { this.topologies = new ArrayList(); } this.topologies.add(elem); } public List get_topologies() { return this.topologies; } public void set_topologies(List topologies) { this.topologies = topologies; } public void unset_topologies() { this.topologies = null; } /** Returns true if field topologies is set (has been assigned a value) and false otherwise */ public boolean is_set_topologies() { return this.topologies != null; } public void set_topologies_isSet(boolean value) { if (!value) { this.topologies = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUPERVISORS: if (value == null) { unset_supervisors(); } else { set_supervisors((List)value); } break; case NIMBUS_UPTIME_SECS: if (value == null) { unset_nimbus_uptime_secs(); } else { set_nimbus_uptime_secs((Integer)value); } break; case TOPOLOGIES: if (value == null) { unset_topologies(); } else { set_topologies((List)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUPERVISORS: return get_supervisors(); case NIMBUS_UPTIME_SECS: return Integer.valueOf(get_nimbus_uptime_secs()); case TOPOLOGIES: return get_topologies(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUPERVISORS: return is_set_supervisors(); case NIMBUS_UPTIME_SECS: return is_set_nimbus_uptime_secs(); case TOPOLOGIES: return is_set_topologies(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof ClusterSummary) return this.equals((ClusterSummary)that); return false; } public boolean equals(ClusterSummary that) { if (that == null) return false; boolean this_present_supervisors = true && this.is_set_supervisors(); boolean that_present_supervisors = true && that.is_set_supervisors(); if (this_present_supervisors || that_present_supervisors) { if (!(this_present_supervisors && that_present_supervisors)) return false; if (!this.supervisors.equals(that.supervisors)) return false; } boolean this_present_nimbus_uptime_secs = true; boolean that_present_nimbus_uptime_secs = true; if (this_present_nimbus_uptime_secs || that_present_nimbus_uptime_secs) { if (!(this_present_nimbus_uptime_secs && that_present_nimbus_uptime_secs)) return false; if (this.nimbus_uptime_secs != that.nimbus_uptime_secs) return false; } boolean this_present_topologies = true && this.is_set_topologies(); boolean that_present_topologies = true && that.is_set_topologies(); if (this_present_topologies || that_present_topologies) { if (!(this_present_topologies && that_present_topologies)) return false; if (!this.topologies.equals(that.topologies)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_supervisors = true && (is_set_supervisors()); builder.append(present_supervisors); if (present_supervisors) builder.append(supervisors); boolean present_nimbus_uptime_secs = true; builder.append(present_nimbus_uptime_secs); if (present_nimbus_uptime_secs) builder.append(nimbus_uptime_secs); boolean present_topologies = true && (is_set_topologies()); builder.append(present_topologies); if (present_topologies) builder.append(topologies); return builder.toHashCode(); } public int compareTo(ClusterSummary other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; ClusterSummary typedOther = (ClusterSummary)other; lastComparison = Boolean.valueOf(is_set_supervisors()).compareTo(typedOther.is_set_supervisors()); if (lastComparison != 0) { return lastComparison; } if (is_set_supervisors()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.supervisors, typedOther.supervisors); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_nimbus_uptime_secs()).compareTo(typedOther.is_set_nimbus_uptime_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_nimbus_uptime_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.nimbus_uptime_secs, typedOther.nimbus_uptime_secs); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(typedOther.is_set_topologies()); if (lastComparison != 0) { return lastComparison; } if (is_set_topologies()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topologies, typedOther.topologies); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // SUPERVISORS if (field.type == org.apache.thrift7.protocol.TType.LIST) { { org.apache.thrift7.protocol.TList _list37 = iprot.readListBegin(); this.supervisors = new ArrayList(_list37.size); for (int _i38 = 0; _i38 < _list37.size; ++_i38) { SupervisorSummary _elem39; // required _elem39 = new SupervisorSummary(); _elem39.read(iprot); this.supervisors.add(_elem39); } iprot.readListEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // NIMBUS_UPTIME_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.nimbus_uptime_secs = iprot.readI32(); set_nimbus_uptime_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // TOPOLOGIES if (field.type == org.apache.thrift7.protocol.TType.LIST) { { org.apache.thrift7.protocol.TList _list40 = iprot.readListBegin(); this.topologies = new ArrayList(_list40.size); for (int _i41 = 0; _i41 < _list40.size; ++_i41) { TopologySummary _elem42; // required _elem42 = new TopologySummary(); _elem42.read(iprot); this.topologies.add(_elem42); } iprot.readListEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.supervisors != null) { oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.supervisors.size())); for (SupervisorSummary _iter43 : this.supervisors) { _iter43.write(oprot); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC); oprot.writeI32(this.nimbus_uptime_secs); oprot.writeFieldEnd(); if (this.topologies != null) { oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.topologies.size())); for (TopologySummary _iter44 : this.topologies) { _iter44.write(oprot); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("ClusterSummary("); boolean first = true; sb.append("supervisors:"); if (this.supervisors == null) { sb.append("null"); } else { sb.append(this.supervisors); } first = false; if (!first) sb.append(", "); sb.append("nimbus_uptime_secs:"); sb.append(this.nimbus_uptime_secs); first = false; if (!first) sb.append(", "); sb.append("topologies:"); if (this.topologies == null) { sb.append("null"); } else { sb.append(this.topologies); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_supervisors()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'supervisors' is unset! Struct:" + toString()); } if (!is_set_nimbus_uptime_secs()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'nimbus_uptime_secs' is unset! Struct:" + toString()); } if (!is_set_topologies()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'topologies' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ComponentCommon implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ComponentCommon"); private static final org.apache.thrift7.protocol.TField INPUTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("inputs", org.apache.thrift7.protocol.TType.MAP, (short)1); private static final org.apache.thrift7.protocol.TField STREAMS_FIELD_DESC = new org.apache.thrift7.protocol.TField("streams", org.apache.thrift7.protocol.TType.MAP, (short)2); private static final org.apache.thrift7.protocol.TField PARALLELISM_HINT_FIELD_DESC = new org.apache.thrift7.protocol.TField("parallelism_hint", org.apache.thrift7.protocol.TType.I32, (short)3); private static final org.apache.thrift7.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift7.protocol.TField("json_conf", org.apache.thrift7.protocol.TType.STRING, (short)4); private Map inputs; // required private Map streams; // required private int parallelism_hint; // required private String json_conf; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { INPUTS((short)1, "inputs"), STREAMS((short)2, "streams"), PARALLELISM_HINT((short)3, "parallelism_hint"), JSON_CONF((short)4, "json_conf"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // INPUTS return INPUTS; case 2: // STREAMS return STREAMS; case 3: // PARALLELISM_HINT return PARALLELISM_HINT; case 4: // JSON_CONF return JSON_CONF; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __PARALLELISM_HINT_ISSET_ID = 0; private BitSet __isset_bit_vector = new BitSet(1); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.INPUTS, new org.apache.thrift7.meta_data.FieldMetaData("inputs", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, Grouping.class)))); tmpMap.put(_Fields.STREAMS, new org.apache.thrift7.meta_data.FieldMetaData("streams", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StreamInfo.class)))); tmpMap.put(_Fields.PARALLELISM_HINT, new org.apache.thrift7.meta_data.FieldMetaData("parallelism_hint", org.apache.thrift7.TFieldRequirementType.OPTIONAL, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift7.meta_data.FieldMetaData("json_conf", org.apache.thrift7.TFieldRequirementType.OPTIONAL, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ComponentCommon.class, metaDataMap); } public ComponentCommon() { } public ComponentCommon( Map inputs, Map streams) { this(); this.inputs = inputs; this.streams = streams; } /** * Performs a deep copy on other. */ public ComponentCommon(ComponentCommon other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_inputs()) { Map __this__inputs = new HashMap(); for (Map.Entry other_element : other.inputs.entrySet()) { GlobalStreamId other_element_key = other_element.getKey(); Grouping other_element_value = other_element.getValue(); GlobalStreamId __this__inputs_copy_key = new GlobalStreamId(other_element_key); Grouping __this__inputs_copy_value = new Grouping(other_element_value); __this__inputs.put(__this__inputs_copy_key, __this__inputs_copy_value); } this.inputs = __this__inputs; } if (other.is_set_streams()) { Map __this__streams = new HashMap(); for (Map.Entry other_element : other.streams.entrySet()) { String other_element_key = other_element.getKey(); StreamInfo other_element_value = other_element.getValue(); String __this__streams_copy_key = other_element_key; StreamInfo __this__streams_copy_value = new StreamInfo(other_element_value); __this__streams.put(__this__streams_copy_key, __this__streams_copy_value); } this.streams = __this__streams; } this.parallelism_hint = other.parallelism_hint; if (other.is_set_json_conf()) { this.json_conf = other.json_conf; } } public ComponentCommon deepCopy() { return new ComponentCommon(this); } @Override public void clear() { this.inputs = null; this.streams = null; set_parallelism_hint_isSet(false); this.parallelism_hint = 0; this.json_conf = null; } public int get_inputs_size() { return (this.inputs == null) ? 0 : this.inputs.size(); } public void put_to_inputs(GlobalStreamId key, Grouping val) { if (this.inputs == null) { this.inputs = new HashMap(); } this.inputs.put(key, val); } public Map get_inputs() { return this.inputs; } public void set_inputs(Map inputs) { this.inputs = inputs; } public void unset_inputs() { this.inputs = null; } /** Returns true if field inputs is set (has been assigned a value) and false otherwise */ public boolean is_set_inputs() { return this.inputs != null; } public void set_inputs_isSet(boolean value) { if (!value) { this.inputs = null; } } public int get_streams_size() { return (this.streams == null) ? 0 : this.streams.size(); } public void put_to_streams(String key, StreamInfo val) { if (this.streams == null) { this.streams = new HashMap(); } this.streams.put(key, val); } public Map get_streams() { return this.streams; } public void set_streams(Map streams) { this.streams = streams; } public void unset_streams() { this.streams = null; } /** Returns true if field streams is set (has been assigned a value) and false otherwise */ public boolean is_set_streams() { return this.streams != null; } public void set_streams_isSet(boolean value) { if (!value) { this.streams = null; } } public int get_parallelism_hint() { return this.parallelism_hint; } public void set_parallelism_hint(int parallelism_hint) { this.parallelism_hint = parallelism_hint; set_parallelism_hint_isSet(true); } public void unset_parallelism_hint() { __isset_bit_vector.clear(__PARALLELISM_HINT_ISSET_ID); } /** Returns true if field parallelism_hint is set (has been assigned a value) and false otherwise */ public boolean is_set_parallelism_hint() { return __isset_bit_vector.get(__PARALLELISM_HINT_ISSET_ID); } public void set_parallelism_hint_isSet(boolean value) { __isset_bit_vector.set(__PARALLELISM_HINT_ISSET_ID, value); } public String get_json_conf() { return this.json_conf; } public void set_json_conf(String json_conf) { this.json_conf = json_conf; } public void unset_json_conf() { this.json_conf = null; } /** Returns true if field json_conf is set (has been assigned a value) and false otherwise */ public boolean is_set_json_conf() { return this.json_conf != null; } public void set_json_conf_isSet(boolean value) { if (!value) { this.json_conf = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case INPUTS: if (value == null) { unset_inputs(); } else { set_inputs((Map)value); } break; case STREAMS: if (value == null) { unset_streams(); } else { set_streams((Map)value); } break; case PARALLELISM_HINT: if (value == null) { unset_parallelism_hint(); } else { set_parallelism_hint((Integer)value); } break; case JSON_CONF: if (value == null) { unset_json_conf(); } else { set_json_conf((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case INPUTS: return get_inputs(); case STREAMS: return get_streams(); case PARALLELISM_HINT: return Integer.valueOf(get_parallelism_hint()); case JSON_CONF: return get_json_conf(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case INPUTS: return is_set_inputs(); case STREAMS: return is_set_streams(); case PARALLELISM_HINT: return is_set_parallelism_hint(); case JSON_CONF: return is_set_json_conf(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof ComponentCommon) return this.equals((ComponentCommon)that); return false; } public boolean equals(ComponentCommon that) { if (that == null) return false; boolean this_present_inputs = true && this.is_set_inputs(); boolean that_present_inputs = true && that.is_set_inputs(); if (this_present_inputs || that_present_inputs) { if (!(this_present_inputs && that_present_inputs)) return false; if (!this.inputs.equals(that.inputs)) return false; } boolean this_present_streams = true && this.is_set_streams(); boolean that_present_streams = true && that.is_set_streams(); if (this_present_streams || that_present_streams) { if (!(this_present_streams && that_present_streams)) return false; if (!this.streams.equals(that.streams)) return false; } boolean this_present_parallelism_hint = true && this.is_set_parallelism_hint(); boolean that_present_parallelism_hint = true && that.is_set_parallelism_hint(); if (this_present_parallelism_hint || that_present_parallelism_hint) { if (!(this_present_parallelism_hint && that_present_parallelism_hint)) return false; if (this.parallelism_hint != that.parallelism_hint) return false; } boolean this_present_json_conf = true && this.is_set_json_conf(); boolean that_present_json_conf = true && that.is_set_json_conf(); if (this_present_json_conf || that_present_json_conf) { if (!(this_present_json_conf && that_present_json_conf)) return false; if (!this.json_conf.equals(that.json_conf)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_inputs = true && (is_set_inputs()); builder.append(present_inputs); if (present_inputs) builder.append(inputs); boolean present_streams = true && (is_set_streams()); builder.append(present_streams); if (present_streams) builder.append(streams); boolean present_parallelism_hint = true && (is_set_parallelism_hint()); builder.append(present_parallelism_hint); if (present_parallelism_hint) builder.append(parallelism_hint); boolean present_json_conf = true && (is_set_json_conf()); builder.append(present_json_conf); if (present_json_conf) builder.append(json_conf); return builder.toHashCode(); } public int compareTo(ComponentCommon other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; ComponentCommon typedOther = (ComponentCommon)other; lastComparison = Boolean.valueOf(is_set_inputs()).compareTo(typedOther.is_set_inputs()); if (lastComparison != 0) { return lastComparison; } if (is_set_inputs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.inputs, typedOther.inputs); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_streams()).compareTo(typedOther.is_set_streams()); if (lastComparison != 0) { return lastComparison; } if (is_set_streams()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.streams, typedOther.streams); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_parallelism_hint()).compareTo(typedOther.is_set_parallelism_hint()); if (lastComparison != 0) { return lastComparison; } if (is_set_parallelism_hint()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.parallelism_hint, typedOther.parallelism_hint); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_json_conf()).compareTo(typedOther.is_set_json_conf()); if (lastComparison != 0) { return lastComparison; } if (is_set_json_conf()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.json_conf, typedOther.json_conf); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // INPUTS if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map12 = iprot.readMapBegin(); this.inputs = new HashMap(2*_map12.size); for (int _i13 = 0; _i13 < _map12.size; ++_i13) { GlobalStreamId _key14; // required Grouping _val15; // required _key14 = new GlobalStreamId(); _key14.read(iprot); _val15 = new Grouping(); _val15.read(iprot); this.inputs.put(_key14, _val15); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // STREAMS if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map16 = iprot.readMapBegin(); this.streams = new HashMap(2*_map16.size); for (int _i17 = 0; _i17 < _map16.size; ++_i17) { String _key18; // required StreamInfo _val19; // required _key18 = iprot.readString(); _val19 = new StreamInfo(); _val19.read(iprot); this.streams.put(_key18, _val19); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // PARALLELISM_HINT if (field.type == org.apache.thrift7.protocol.TType.I32) { this.parallelism_hint = iprot.readI32(); set_parallelism_hint_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // JSON_CONF if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.json_conf = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.inputs != null) { oprot.writeFieldBegin(INPUTS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.STRUCT, this.inputs.size())); for (Map.Entry _iter20 : this.inputs.entrySet()) { _iter20.getKey().write(oprot); _iter20.getValue().write(oprot); } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.streams != null) { oprot.writeFieldBegin(STREAMS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.streams.size())); for (Map.Entry _iter21 : this.streams.entrySet()) { oprot.writeString(_iter21.getKey()); _iter21.getValue().write(oprot); } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (is_set_parallelism_hint()) { oprot.writeFieldBegin(PARALLELISM_HINT_FIELD_DESC); oprot.writeI32(this.parallelism_hint); oprot.writeFieldEnd(); } if (this.json_conf != null) { if (is_set_json_conf()) { oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); oprot.writeString(this.json_conf); oprot.writeFieldEnd(); } } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("ComponentCommon("); boolean first = true; sb.append("inputs:"); if (this.inputs == null) { sb.append("null"); } else { sb.append(this.inputs); } first = false; if (!first) sb.append(", "); sb.append("streams:"); if (this.streams == null) { sb.append("null"); } else { sb.append(this.streams); } first = false; if (is_set_parallelism_hint()) { if (!first) sb.append(", "); sb.append("parallelism_hint:"); sb.append(this.parallelism_hint); first = false; } if (is_set_json_conf()) { if (!first) sb.append(", "); sb.append("json_conf:"); if (this.json_conf == null) { sb.append("null"); } else { sb.append(this.json_conf); } first = false; } sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_inputs()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'inputs' is unset! Struct:" + toString()); } if (!is_set_streams()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'streams' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ComponentObject.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ComponentObject extends org.apache.thrift7.TUnion { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ComponentObject"); private static final org.apache.thrift7.protocol.TField SERIALIZED_JAVA_FIELD_DESC = new org.apache.thrift7.protocol.TField("serialized_java", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField SHELL_FIELD_DESC = new org.apache.thrift7.protocol.TField("shell", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private static final org.apache.thrift7.protocol.TField JAVA_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("java_object", org.apache.thrift7.protocol.TType.STRUCT, (short)3); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SERIALIZED_JAVA((short)1, "serialized_java"), SHELL((short)2, "shell"), JAVA_OBJECT((short)3, "java_object"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // SERIALIZED_JAVA return SERIALIZED_JAVA; case 2: // SHELL return SHELL; case 3: // JAVA_OBJECT return JAVA_OBJECT; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SERIALIZED_JAVA, new org.apache.thrift7.meta_data.FieldMetaData("serialized_java", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); tmpMap.put(_Fields.SHELL, new org.apache.thrift7.meta_data.FieldMetaData("shell", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ShellComponent.class))); tmpMap.put(_Fields.JAVA_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("java_object", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, JavaObject.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ComponentObject.class, metaDataMap); } public ComponentObject() { super(); } public ComponentObject(_Fields setField, Object value) { super(setField, value); } public ComponentObject(ComponentObject other) { super(other); } public ComponentObject deepCopy() { return new ComponentObject(this); } public static ComponentObject serialized_java(ByteBuffer value) { ComponentObject x = new ComponentObject(); x.set_serialized_java(value); return x; } public static ComponentObject serialized_java(byte[] value) { ComponentObject x = new ComponentObject(); x.set_serialized_java(ByteBuffer.wrap(value)); return x; } public static ComponentObject shell(ShellComponent value) { ComponentObject x = new ComponentObject(); x.set_shell(value); return x; } public static ComponentObject java_object(JavaObject value) { ComponentObject x = new ComponentObject(); x.set_java_object(value); return x; } @Override protected void checkType(_Fields setField, Object value) throws ClassCastException { switch (setField) { case SERIALIZED_JAVA: if (value instanceof ByteBuffer) { break; } throw new ClassCastException("Was expecting value of type ByteBuffer for field 'serialized_java', but got " + value.getClass().getSimpleName()); case SHELL: if (value instanceof ShellComponent) { break; } throw new ClassCastException("Was expecting value of type ShellComponent for field 'shell', but got " + value.getClass().getSimpleName()); case JAVA_OBJECT: if (value instanceof JavaObject) { break; } throw new ClassCastException("Was expecting value of type JavaObject for field 'java_object', but got " + value.getClass().getSimpleName()); default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { _Fields setField = _Fields.findByThriftId(field.id); if (setField != null) { switch (setField) { case SERIALIZED_JAVA: if (field.type == SERIALIZED_JAVA_FIELD_DESC.type) { ByteBuffer serialized_java; serialized_java = iprot.readBinary(); return serialized_java; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case SHELL: if (field.type == SHELL_FIELD_DESC.type) { ShellComponent shell; shell = new ShellComponent(); shell.read(iprot); return shell; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case JAVA_OBJECT: if (field.type == JAVA_OBJECT_FIELD_DESC.type) { JavaObject java_object; java_object = new JavaObject(); java_object.read(iprot); return java_object; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } default: throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } } @Override protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { switch (setField_) { case SERIALIZED_JAVA: ByteBuffer serialized_java = (ByteBuffer)value_; oprot.writeBinary(serialized_java); return; case SHELL: ShellComponent shell = (ShellComponent)value_; shell.write(oprot); return; case JAVA_OBJECT: JavaObject java_object = (JavaObject)value_; java_object.write(oprot); return; default: throw new IllegalStateException("Cannot write union with unknown field " + setField_); } } @Override protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { switch (setField) { case SERIALIZED_JAVA: return SERIALIZED_JAVA_FIELD_DESC; case SHELL: return SHELL_FIELD_DESC; case JAVA_OBJECT: return JAVA_OBJECT_FIELD_DESC; default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected org.apache.thrift7.protocol.TStruct getStructDesc() { return STRUCT_DESC; } @Override protected _Fields enumForId(short id) { return _Fields.findByThriftIdOrThrow(id); } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public byte[] get_serialized_java() { set_serialized_java(org.apache.thrift7.TBaseHelper.rightSize(buffer_for_serialized_java())); ByteBuffer b = buffer_for_serialized_java(); return b == null ? null : b.array(); } public ByteBuffer buffer_for_serialized_java() { if (getSetField() == _Fields.SERIALIZED_JAVA) { return (ByteBuffer)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'serialized_java' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_serialized_java(byte[] value) { set_serialized_java(ByteBuffer.wrap(value)); } public void set_serialized_java(ByteBuffer value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.SERIALIZED_JAVA; value_ = value; } public ShellComponent get_shell() { if (getSetField() == _Fields.SHELL) { return (ShellComponent)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'shell' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_shell(ShellComponent value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.SHELL; value_ = value; } public JavaObject get_java_object() { if (getSetField() == _Fields.JAVA_OBJECT) { return (JavaObject)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'java_object' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_java_object(JavaObject value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.JAVA_OBJECT; value_ = value; } public boolean is_set_serialized_java() { return setField_ == _Fields.SERIALIZED_JAVA; } public boolean is_set_shell() { return setField_ == _Fields.SHELL; } public boolean is_set_java_object() { return setField_ == _Fields.JAVA_OBJECT; } public boolean equals(Object other) { if (other instanceof ComponentObject) { return equals((ComponentObject)other); } else { return false; } } public boolean equals(ComponentObject other) { return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); } @Override public int compareTo(ComponentObject other) { int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); if (lastComparison == 0) { return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); } return lastComparison; } @Override public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(); hcb.append(this.getClass().getName()); org.apache.thrift7.TFieldIdEnum setField = getSetField(); if (setField != null) { hcb.append(setField.getThriftFieldId()); Object value = getFieldValue(); if (value instanceof org.apache.thrift7.TEnum) { hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); } else { hcb.append(value); } } return hcb.toHashCode(); } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DRPCExecutionException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("DRPCExecutionException"); private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); private String msg; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { MSG((short)1, "msg"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // MSG return MSG; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(DRPCExecutionException.class, metaDataMap); } public DRPCExecutionException() { } public DRPCExecutionException( String msg) { this(); this.msg = msg; } /** * Performs a deep copy on other. */ public DRPCExecutionException(DRPCExecutionException other) { if (other.is_set_msg()) { this.msg = other.msg; } } public DRPCExecutionException deepCopy() { return new DRPCExecutionException(this); } @Override public void clear() { this.msg = null; } public String get_msg() { return this.msg; } public void set_msg(String msg) { this.msg = msg; } public void unset_msg() { this.msg = null; } /** Returns true if field msg is set (has been assigned a value) and false otherwise */ public boolean is_set_msg() { return this.msg != null; } public void set_msg_isSet(boolean value) { if (!value) { this.msg = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case MSG: if (value == null) { unset_msg(); } else { set_msg((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case MSG: return get_msg(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case MSG: return is_set_msg(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof DRPCExecutionException) return this.equals((DRPCExecutionException)that); return false; } public boolean equals(DRPCExecutionException that) { if (that == null) return false; boolean this_present_msg = true && this.is_set_msg(); boolean that_present_msg = true && that.is_set_msg(); if (this_present_msg || that_present_msg) { if (!(this_present_msg && that_present_msg)) return false; if (!this.msg.equals(that.msg)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_msg = true && (is_set_msg()); builder.append(present_msg); if (present_msg) builder.append(msg); return builder.toHashCode(); } public int compareTo(DRPCExecutionException other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; DRPCExecutionException typedOther = (DRPCExecutionException)other; lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); if (lastComparison != 0) { return lastComparison; } if (is_set_msg()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // MSG if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.msg = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.msg != null) { oprot.writeFieldBegin(MSG_FIELD_DESC); oprot.writeString(this.msg); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("DRPCExecutionException("); boolean first = true; sb.append("msg:"); if (this.msg == null) { sb.append("null"); } else { sb.append(this.msg); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_msg()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DRPCRequest implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("DRPCRequest"); private static final org.apache.thrift7.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift7.protocol.TField("func_args", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField REQUEST_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("request_id", org.apache.thrift7.protocol.TType.STRING, (short)2); private String func_args; // required private String request_id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { FUNC_ARGS((short)1, "func_args"), REQUEST_ID((short)2, "request_id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // FUNC_ARGS return FUNC_ARGS; case 2: // REQUEST_ID return REQUEST_ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift7.meta_data.FieldMetaData("func_args", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.REQUEST_ID, new org.apache.thrift7.meta_data.FieldMetaData("request_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(DRPCRequest.class, metaDataMap); } public DRPCRequest() { } public DRPCRequest( String func_args, String request_id) { this(); this.func_args = func_args; this.request_id = request_id; } /** * Performs a deep copy on other. */ public DRPCRequest(DRPCRequest other) { if (other.is_set_func_args()) { this.func_args = other.func_args; } if (other.is_set_request_id()) { this.request_id = other.request_id; } } public DRPCRequest deepCopy() { return new DRPCRequest(this); } @Override public void clear() { this.func_args = null; this.request_id = null; } public String get_func_args() { return this.func_args; } public void set_func_args(String func_args) { this.func_args = func_args; } public void unset_func_args() { this.func_args = null; } /** Returns true if field func_args is set (has been assigned a value) and false otherwise */ public boolean is_set_func_args() { return this.func_args != null; } public void set_func_args_isSet(boolean value) { if (!value) { this.func_args = null; } } public String get_request_id() { return this.request_id; } public void set_request_id(String request_id) { this.request_id = request_id; } public void unset_request_id() { this.request_id = null; } /** Returns true if field request_id is set (has been assigned a value) and false otherwise */ public boolean is_set_request_id() { return this.request_id != null; } public void set_request_id_isSet(boolean value) { if (!value) { this.request_id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case FUNC_ARGS: if (value == null) { unset_func_args(); } else { set_func_args((String)value); } break; case REQUEST_ID: if (value == null) { unset_request_id(); } else { set_request_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case FUNC_ARGS: return get_func_args(); case REQUEST_ID: return get_request_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case FUNC_ARGS: return is_set_func_args(); case REQUEST_ID: return is_set_request_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof DRPCRequest) return this.equals((DRPCRequest)that); return false; } public boolean equals(DRPCRequest that) { if (that == null) return false; boolean this_present_func_args = true && this.is_set_func_args(); boolean that_present_func_args = true && that.is_set_func_args(); if (this_present_func_args || that_present_func_args) { if (!(this_present_func_args && that_present_func_args)) return false; if (!this.func_args.equals(that.func_args)) return false; } boolean this_present_request_id = true && this.is_set_request_id(); boolean that_present_request_id = true && that.is_set_request_id(); if (this_present_request_id || that_present_request_id) { if (!(this_present_request_id && that_present_request_id)) return false; if (!this.request_id.equals(that.request_id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_func_args = true && (is_set_func_args()); builder.append(present_func_args); if (present_func_args) builder.append(func_args); boolean present_request_id = true && (is_set_request_id()); builder.append(present_request_id); if (present_request_id) builder.append(request_id); return builder.toHashCode(); } public int compareTo(DRPCRequest other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; DRPCRequest typedOther = (DRPCRequest)other; lastComparison = Boolean.valueOf(is_set_func_args()).compareTo(typedOther.is_set_func_args()); if (lastComparison != 0) { return lastComparison; } if (is_set_func_args()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.func_args, typedOther.func_args); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_request_id()).compareTo(typedOther.is_set_request_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_request_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.request_id, typedOther.request_id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // FUNC_ARGS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.func_args = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // REQUEST_ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.request_id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.func_args != null) { oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC); oprot.writeString(this.func_args); oprot.writeFieldEnd(); } if (this.request_id != null) { oprot.writeFieldBegin(REQUEST_ID_FIELD_DESC); oprot.writeString(this.request_id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("DRPCRequest("); boolean first = true; sb.append("func_args:"); if (this.func_args == null) { sb.append("null"); } else { sb.append(this.func_args); } first = false; if (!first) sb.append(", "); sb.append("request_id:"); if (this.request_id == null) { sb.append("null"); } else { sb.append(this.request_id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_func_args()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'func_args' is unset! Struct:" + toString()); } if (!is_set_request_id()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'request_id' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DistributedRPC { public interface Iface { public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift7.TException; } public interface AsyncIface { public void execute(String functionName, String funcArgs, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; } public static class Client extends org.apache.thrift7.TServiceClient implements Iface { public static class Factory implements org.apache.thrift7.TServiceClientFactory { public Factory() {} public Client getClient(org.apache.thrift7.protocol.TProtocol prot) { return new Client(prot); } public Client getClient(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { return new Client(iprot, oprot); } } public Client(org.apache.thrift7.protocol.TProtocol prot) { super(prot, prot); } public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { super(iprot, oprot); } public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift7.TException { send_execute(functionName, funcArgs); return recv_execute(); } public void send_execute(String functionName, String funcArgs) throws org.apache.thrift7.TException { execute_args args = new execute_args(); args.set_functionName(functionName); args.set_funcArgs(funcArgs); sendBase("execute", args); } public String recv_execute() throws DRPCExecutionException, org.apache.thrift7.TException { execute_result result = new execute_result(); receiveBase(result, "execute"); if (result.is_set_success()) { return result.success; } if (result.e != null) { throw result.e; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "execute failed: unknown result"); } } public static class AsyncClient extends org.apache.thrift7.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift7.async.TAsyncClientFactory { private org.apache.thrift7.async.TAsyncClientManager clientManager; private org.apache.thrift7.protocol.TProtocolFactory protocolFactory; public Factory(org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.protocol.TProtocolFactory protocolFactory) { this.clientManager = clientManager; this.protocolFactory = protocolFactory; } public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNonblockingTransport transport) { return new AsyncClient(protocolFactory, clientManager, transport); } } public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.transport.TNonblockingTransport transport) { super(protocolFactory, clientManager, transport); } public void execute(String functionName, String funcArgs, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); execute_call method_call = new execute_call(functionName, funcArgs, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class execute_call extends org.apache.thrift7.async.TAsyncMethodCall { private String functionName; private String funcArgs; public execute_call(String functionName, String funcArgs, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.functionName = functionName; this.funcArgs = funcArgs; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("execute", org.apache.thrift7.protocol.TMessageType.CALL, 0)); execute_args args = new execute_args(); args.set_functionName(functionName); args.set_funcArgs(funcArgs); args.write(prot); prot.writeMessageEnd(); } public String getResult() throws DRPCExecutionException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_execute(); } } } public static class Processor extends org.apache.thrift7.TBaseProcessor implements org.apache.thrift7.TProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); public Processor(I iface) { super(iface, getProcessMap(new HashMap>())); } protected Processor(I iface, Map> processMap) { super(iface, getProcessMap(processMap)); } private static Map> getProcessMap(Map> processMap) { processMap.put("execute", new execute()); return processMap; } private static class execute extends org.apache.thrift7.ProcessFunction { public execute() { super("execute"); } protected execute_args getEmptyArgsInstance() { return new execute_args(); } protected execute_result getResult(I iface, execute_args args) throws org.apache.thrift7.TException { execute_result result = new execute_result(); try { result.success = iface.execute(args.functionName, args.funcArgs); } catch (DRPCExecutionException e) { result.e = e; } return result; } } } public static class execute_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("execute_args"); private static final org.apache.thrift7.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("functionName", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift7.protocol.TField("funcArgs", org.apache.thrift7.protocol.TType.STRING, (short)2); private String functionName; // required private String funcArgs; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { FUNCTION_NAME((short)1, "functionName"), FUNC_ARGS((short)2, "funcArgs"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // FUNCTION_NAME return FUNCTION_NAME; case 2: // FUNC_ARGS return FUNC_ARGS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift7.meta_data.FieldMetaData("functionName", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift7.meta_data.FieldMetaData("funcArgs", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(execute_args.class, metaDataMap); } public execute_args() { } public execute_args( String functionName, String funcArgs) { this(); this.functionName = functionName; this.funcArgs = funcArgs; } /** * Performs a deep copy on other. */ public execute_args(execute_args other) { if (other.is_set_functionName()) { this.functionName = other.functionName; } if (other.is_set_funcArgs()) { this.funcArgs = other.funcArgs; } } public execute_args deepCopy() { return new execute_args(this); } @Override public void clear() { this.functionName = null; this.funcArgs = null; } public String get_functionName() { return this.functionName; } public void set_functionName(String functionName) { this.functionName = functionName; } public void unset_functionName() { this.functionName = null; } /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ public boolean is_set_functionName() { return this.functionName != null; } public void set_functionName_isSet(boolean value) { if (!value) { this.functionName = null; } } public String get_funcArgs() { return this.funcArgs; } public void set_funcArgs(String funcArgs) { this.funcArgs = funcArgs; } public void unset_funcArgs() { this.funcArgs = null; } /** Returns true if field funcArgs is set (has been assigned a value) and false otherwise */ public boolean is_set_funcArgs() { return this.funcArgs != null; } public void set_funcArgs_isSet(boolean value) { if (!value) { this.funcArgs = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case FUNCTION_NAME: if (value == null) { unset_functionName(); } else { set_functionName((String)value); } break; case FUNC_ARGS: if (value == null) { unset_funcArgs(); } else { set_funcArgs((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case FUNCTION_NAME: return get_functionName(); case FUNC_ARGS: return get_funcArgs(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case FUNCTION_NAME: return is_set_functionName(); case FUNC_ARGS: return is_set_funcArgs(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof execute_args) return this.equals((execute_args)that); return false; } public boolean equals(execute_args that) { if (that == null) return false; boolean this_present_functionName = true && this.is_set_functionName(); boolean that_present_functionName = true && that.is_set_functionName(); if (this_present_functionName || that_present_functionName) { if (!(this_present_functionName && that_present_functionName)) return false; if (!this.functionName.equals(that.functionName)) return false; } boolean this_present_funcArgs = true && this.is_set_funcArgs(); boolean that_present_funcArgs = true && that.is_set_funcArgs(); if (this_present_funcArgs || that_present_funcArgs) { if (!(this_present_funcArgs && that_present_funcArgs)) return false; if (!this.funcArgs.equals(that.funcArgs)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_functionName = true && (is_set_functionName()); builder.append(present_functionName); if (present_functionName) builder.append(functionName); boolean present_funcArgs = true && (is_set_funcArgs()); builder.append(present_funcArgs); if (present_funcArgs) builder.append(funcArgs); return builder.toHashCode(); } public int compareTo(execute_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; execute_args typedOther = (execute_args)other; lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(typedOther.is_set_functionName()); if (lastComparison != 0) { return lastComparison; } if (is_set_functionName()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.functionName, typedOther.functionName); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_funcArgs()).compareTo(typedOther.is_set_funcArgs()); if (lastComparison != 0) { return lastComparison; } if (is_set_funcArgs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.funcArgs, typedOther.funcArgs); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // FUNCTION_NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.functionName = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // FUNC_ARGS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.funcArgs = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.functionName != null) { oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); oprot.writeString(this.functionName); oprot.writeFieldEnd(); } if (this.funcArgs != null) { oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC); oprot.writeString(this.funcArgs); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("execute_args("); boolean first = true; sb.append("functionName:"); if (this.functionName == null) { sb.append("null"); } else { sb.append(this.functionName); } first = false; if (!first) sb.append(", "); sb.append("funcArgs:"); if (this.funcArgs == null) { sb.append("null"); } else { sb.append(this.funcArgs); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class execute_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("execute_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private String success; // required private DRPCExecutionException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"), E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(execute_result.class, metaDataMap); } public execute_result() { } public execute_result( String success, DRPCExecutionException e) { this(); this.success = success; this.e = e; } /** * Performs a deep copy on other. */ public execute_result(execute_result other) { if (other.is_set_success()) { this.success = other.success; } if (other.is_set_e()) { this.e = new DRPCExecutionException(other.e); } } public execute_result deepCopy() { return new execute_result(this); } @Override public void clear() { this.success = null; this.e = null; } public String get_success() { return this.success; } public void set_success(String success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public DRPCExecutionException get_e() { return this.e; } public void set_e(DRPCExecutionException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((String)value); } break; case E: if (value == null) { unset_e(); } else { set_e((DRPCExecutionException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof execute_result) return this.equals((execute_result)that); return false; } public boolean equals(execute_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(execute_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; execute_result typedOther = (execute_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.success = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new DRPCExecutionException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); } else if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("execute_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; if (!first) sb.append(", "); sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DistributedRPCInvocations { public interface Iface { public void result(String id, String result) throws org.apache.thrift7.TException; public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift7.TException; public void failRequest(String id) throws org.apache.thrift7.TException; } public interface AsyncIface { public void result(String id, String result, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void fetchRequest(String functionName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void failRequest(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; } public static class Client extends org.apache.thrift7.TServiceClient implements Iface { public static class Factory implements org.apache.thrift7.TServiceClientFactory { public Factory() {} public Client getClient(org.apache.thrift7.protocol.TProtocol prot) { return new Client(prot); } public Client getClient(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { return new Client(iprot, oprot); } } public Client(org.apache.thrift7.protocol.TProtocol prot) { super(prot, prot); } public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { super(iprot, oprot); } public void result(String id, String result) throws org.apache.thrift7.TException { send_result(id, result); recv_result(); } public void send_result(String id, String result) throws org.apache.thrift7.TException { result_args args = new result_args(); args.set_id(id); args.set_result(result); sendBase("result", args); } public void recv_result() throws org.apache.thrift7.TException { result_result result = new result_result(); receiveBase(result, "result"); return; } public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift7.TException { send_fetchRequest(functionName); return recv_fetchRequest(); } public void send_fetchRequest(String functionName) throws org.apache.thrift7.TException { fetchRequest_args args = new fetchRequest_args(); args.set_functionName(functionName); sendBase("fetchRequest", args); } public DRPCRequest recv_fetchRequest() throws org.apache.thrift7.TException { fetchRequest_result result = new fetchRequest_result(); receiveBase(result, "fetchRequest"); if (result.is_set_success()) { return result.success; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result"); } public void failRequest(String id) throws org.apache.thrift7.TException { send_failRequest(id); recv_failRequest(); } public void send_failRequest(String id) throws org.apache.thrift7.TException { failRequest_args args = new failRequest_args(); args.set_id(id); sendBase("failRequest", args); } public void recv_failRequest() throws org.apache.thrift7.TException { failRequest_result result = new failRequest_result(); receiveBase(result, "failRequest"); return; } } public static class AsyncClient extends org.apache.thrift7.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift7.async.TAsyncClientFactory { private org.apache.thrift7.async.TAsyncClientManager clientManager; private org.apache.thrift7.protocol.TProtocolFactory protocolFactory; public Factory(org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.protocol.TProtocolFactory protocolFactory) { this.clientManager = clientManager; this.protocolFactory = protocolFactory; } public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNonblockingTransport transport) { return new AsyncClient(protocolFactory, clientManager, transport); } } public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.transport.TNonblockingTransport transport) { super(protocolFactory, clientManager, transport); } public void result(String id, String result, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); result_call method_call = new result_call(id, result, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class result_call extends org.apache.thrift7.async.TAsyncMethodCall { private String id; private String result; public result_call(String id, String result, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.id = id; this.result = result; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("result", org.apache.thrift7.protocol.TMessageType.CALL, 0)); result_args args = new result_args(); args.set_id(id); args.set_result(result); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_result(); } } public void fetchRequest(String functionName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); fetchRequest_call method_call = new fetchRequest_call(functionName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class fetchRequest_call extends org.apache.thrift7.async.TAsyncMethodCall { private String functionName; public fetchRequest_call(String functionName, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.functionName = functionName; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("fetchRequest", org.apache.thrift7.protocol.TMessageType.CALL, 0)); fetchRequest_args args = new fetchRequest_args(); args.set_functionName(functionName); args.write(prot); prot.writeMessageEnd(); } public DRPCRequest getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_fetchRequest(); } } public void failRequest(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); failRequest_call method_call = new failRequest_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class failRequest_call extends org.apache.thrift7.async.TAsyncMethodCall { private String id; public failRequest_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.id = id; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("failRequest", org.apache.thrift7.protocol.TMessageType.CALL, 0)); failRequest_args args = new failRequest_args(); args.set_id(id); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_failRequest(); } } } public static class Processor extends org.apache.thrift7.TBaseProcessor implements org.apache.thrift7.TProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); public Processor(I iface) { super(iface, getProcessMap(new HashMap>())); } protected Processor(I iface, Map> processMap) { super(iface, getProcessMap(processMap)); } private static Map> getProcessMap(Map> processMap) { processMap.put("result", new result()); processMap.put("fetchRequest", new fetchRequest()); processMap.put("failRequest", new failRequest()); return processMap; } private static class result extends org.apache.thrift7.ProcessFunction { public result() { super("result"); } protected result_args getEmptyArgsInstance() { return new result_args(); } protected result_result getResult(I iface, result_args args) throws org.apache.thrift7.TException { result_result result = new result_result(); iface.result(args.id, args.result); return result; } } private static class fetchRequest extends org.apache.thrift7.ProcessFunction { public fetchRequest() { super("fetchRequest"); } protected fetchRequest_args getEmptyArgsInstance() { return new fetchRequest_args(); } protected fetchRequest_result getResult(I iface, fetchRequest_args args) throws org.apache.thrift7.TException { fetchRequest_result result = new fetchRequest_result(); result.success = iface.fetchRequest(args.functionName); return result; } } private static class failRequest extends org.apache.thrift7.ProcessFunction { public failRequest() { super("failRequest"); } protected failRequest_args getEmptyArgsInstance() { return new failRequest_args(); } protected failRequest_result getResult(I iface, failRequest_args args) throws org.apache.thrift7.TException { failRequest_result result = new failRequest_result(); iface.failRequest(args.id); return result; } } } public static class result_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("result_args"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField RESULT_FIELD_DESC = new org.apache.thrift7.protocol.TField("result", org.apache.thrift7.protocol.TType.STRING, (short)2); private String id; // required private String result; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"), RESULT((short)2, "result"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; case 2: // RESULT return RESULT; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.RESULT, new org.apache.thrift7.meta_data.FieldMetaData("result", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(result_args.class, metaDataMap); } public result_args() { } public result_args( String id, String result) { this(); this.id = id; this.result = result; } /** * Performs a deep copy on other. */ public result_args(result_args other) { if (other.is_set_id()) { this.id = other.id; } if (other.is_set_result()) { this.result = other.result; } } public result_args deepCopy() { return new result_args(this); } @Override public void clear() { this.id = null; this.result = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public String get_result() { return this.result; } public void set_result(String result) { this.result = result; } public void unset_result() { this.result = null; } /** Returns true if field result is set (has been assigned a value) and false otherwise */ public boolean is_set_result() { return this.result != null; } public void set_result_isSet(boolean value) { if (!value) { this.result = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; case RESULT: if (value == null) { unset_result(); } else { set_result((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); case RESULT: return get_result(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); case RESULT: return is_set_result(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof result_args) return this.equals((result_args)that); return false; } public boolean equals(result_args that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } boolean this_present_result = true && this.is_set_result(); boolean that_present_result = true && that.is_set_result(); if (this_present_result || that_present_result) { if (!(this_present_result && that_present_result)) return false; if (!this.result.equals(that.result)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); boolean present_result = true && (is_set_result()); builder.append(present_result); if (present_result) builder.append(result); return builder.toHashCode(); } public int compareTo(result_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; result_args typedOther = (result_args)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_result()).compareTo(typedOther.is_set_result()); if (lastComparison != 0) { return lastComparison; } if (is_set_result()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.result, typedOther.result); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // RESULT if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.result = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } if (this.result != null) { oprot.writeFieldBegin(RESULT_FIELD_DESC); oprot.writeString(this.result); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("result_args("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; if (!first) sb.append(", "); sb.append("result:"); if (this.result == null) { sb.append("null"); } else { sb.append(this.result); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class result_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("result_result"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(result_result.class, metaDataMap); } public result_result() { } /** * Performs a deep copy on other. */ public result_result(result_result other) { } public result_result deepCopy() { return new result_result(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof result_result) return this.equals((result_result)that); return false; } public boolean equals(result_result that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(result_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; result_result typedOther = (result_result)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("result_result("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class fetchRequest_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("fetchRequest_args"); private static final org.apache.thrift7.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("functionName", org.apache.thrift7.protocol.TType.STRING, (short)1); private String functionName; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { FUNCTION_NAME((short)1, "functionName"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // FUNCTION_NAME return FUNCTION_NAME; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift7.meta_data.FieldMetaData("functionName", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_args.class, metaDataMap); } public fetchRequest_args() { } public fetchRequest_args( String functionName) { this(); this.functionName = functionName; } /** * Performs a deep copy on other. */ public fetchRequest_args(fetchRequest_args other) { if (other.is_set_functionName()) { this.functionName = other.functionName; } } public fetchRequest_args deepCopy() { return new fetchRequest_args(this); } @Override public void clear() { this.functionName = null; } public String get_functionName() { return this.functionName; } public void set_functionName(String functionName) { this.functionName = functionName; } public void unset_functionName() { this.functionName = null; } /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ public boolean is_set_functionName() { return this.functionName != null; } public void set_functionName_isSet(boolean value) { if (!value) { this.functionName = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case FUNCTION_NAME: if (value == null) { unset_functionName(); } else { set_functionName((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case FUNCTION_NAME: return get_functionName(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case FUNCTION_NAME: return is_set_functionName(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof fetchRequest_args) return this.equals((fetchRequest_args)that); return false; } public boolean equals(fetchRequest_args that) { if (that == null) return false; boolean this_present_functionName = true && this.is_set_functionName(); boolean that_present_functionName = true && that.is_set_functionName(); if (this_present_functionName || that_present_functionName) { if (!(this_present_functionName && that_present_functionName)) return false; if (!this.functionName.equals(that.functionName)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_functionName = true && (is_set_functionName()); builder.append(present_functionName); if (present_functionName) builder.append(functionName); return builder.toHashCode(); } public int compareTo(fetchRequest_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; fetchRequest_args typedOther = (fetchRequest_args)other; lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(typedOther.is_set_functionName()); if (lastComparison != 0) { return lastComparison; } if (is_set_functionName()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.functionName, typedOther.functionName); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // FUNCTION_NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.functionName = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.functionName != null) { oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); oprot.writeString(this.functionName); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("fetchRequest_args("); boolean first = true; sb.append("functionName:"); if (this.functionName == null) { sb.append("null"); } else { sb.append(this.functionName); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class fetchRequest_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("fetchRequest_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); private DRPCRequest success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, DRPCRequest.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_result.class, metaDataMap); } public fetchRequest_result() { } public fetchRequest_result( DRPCRequest success) { this(); this.success = success; } /** * Performs a deep copy on other. */ public fetchRequest_result(fetchRequest_result other) { if (other.is_set_success()) { this.success = new DRPCRequest(other.success); } } public fetchRequest_result deepCopy() { return new fetchRequest_result(this); } @Override public void clear() { this.success = null; } public DRPCRequest get_success() { return this.success; } public void set_success(DRPCRequest success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((DRPCRequest)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof fetchRequest_result) return this.equals((fetchRequest_result)that); return false; } public boolean equals(fetchRequest_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); return builder.toHashCode(); } public int compareTo(fetchRequest_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; fetchRequest_result typedOther = (fetchRequest_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.success = new DRPCRequest(); this.success.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); this.success.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("fetchRequest_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class failRequest_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("failRequest_args"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private String id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_args.class, metaDataMap); } public failRequest_args() { } public failRequest_args( String id) { this(); this.id = id; } /** * Performs a deep copy on other. */ public failRequest_args(failRequest_args other) { if (other.is_set_id()) { this.id = other.id; } } public failRequest_args deepCopy() { return new failRequest_args(this); } @Override public void clear() { this.id = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof failRequest_args) return this.equals((failRequest_args)that); return false; } public boolean equals(failRequest_args that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); return builder.toHashCode(); } public int compareTo(failRequest_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; failRequest_args typedOther = (failRequest_args)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("failRequest_args("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class failRequest_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("failRequest_result"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_result.class, metaDataMap); } public failRequest_result() { } /** * Performs a deep copy on other. */ public failRequest_result(failRequest_result other) { } public failRequest_result deepCopy() { return new failRequest_result(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof failRequest_result) return this.equals((failRequest_result)that); return false; } public boolean equals(failRequest_result that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(failRequest_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; failRequest_result typedOther = (failRequest_result)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("failRequest_result("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ErrorInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ErrorInfo"); private static final org.apache.thrift7.protocol.TField ERROR_FIELD_DESC = new org.apache.thrift7.protocol.TField("error", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField ERROR_TIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("error_time_secs", org.apache.thrift7.protocol.TType.I32, (short)2); private String error; // required private int error_time_secs; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ERROR((short)1, "error"), ERROR_TIME_SECS((short)2, "error_time_secs"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ERROR return ERROR; case 2: // ERROR_TIME_SECS return ERROR_TIME_SECS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __ERROR_TIME_SECS_ISSET_ID = 0; private BitSet __isset_bit_vector = new BitSet(1); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ERROR, new org.apache.thrift7.meta_data.FieldMetaData("error", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.ERROR_TIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("error_time_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ErrorInfo.class, metaDataMap); } public ErrorInfo() { } public ErrorInfo( String error, int error_time_secs) { this(); this.error = error; this.error_time_secs = error_time_secs; set_error_time_secs_isSet(true); } /** * Performs a deep copy on other. */ public ErrorInfo(ErrorInfo other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_error()) { this.error = other.error; } this.error_time_secs = other.error_time_secs; } public ErrorInfo deepCopy() { return new ErrorInfo(this); } @Override public void clear() { this.error = null; set_error_time_secs_isSet(false); this.error_time_secs = 0; } public String get_error() { return this.error; } public void set_error(String error) { this.error = error; } public void unset_error() { this.error = null; } /** Returns true if field error is set (has been assigned a value) and false otherwise */ public boolean is_set_error() { return this.error != null; } public void set_error_isSet(boolean value) { if (!value) { this.error = null; } } public int get_error_time_secs() { return this.error_time_secs; } public void set_error_time_secs(int error_time_secs) { this.error_time_secs = error_time_secs; set_error_time_secs_isSet(true); } public void unset_error_time_secs() { __isset_bit_vector.clear(__ERROR_TIME_SECS_ISSET_ID); } /** Returns true if field error_time_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_error_time_secs() { return __isset_bit_vector.get(__ERROR_TIME_SECS_ISSET_ID); } public void set_error_time_secs_isSet(boolean value) { __isset_bit_vector.set(__ERROR_TIME_SECS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { switch (field) { case ERROR: if (value == null) { unset_error(); } else { set_error((String)value); } break; case ERROR_TIME_SECS: if (value == null) { unset_error_time_secs(); } else { set_error_time_secs((Integer)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ERROR: return get_error(); case ERROR_TIME_SECS: return Integer.valueOf(get_error_time_secs()); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ERROR: return is_set_error(); case ERROR_TIME_SECS: return is_set_error_time_secs(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof ErrorInfo) return this.equals((ErrorInfo)that); return false; } public boolean equals(ErrorInfo that) { if (that == null) return false; boolean this_present_error = true && this.is_set_error(); boolean that_present_error = true && that.is_set_error(); if (this_present_error || that_present_error) { if (!(this_present_error && that_present_error)) return false; if (!this.error.equals(that.error)) return false; } boolean this_present_error_time_secs = true; boolean that_present_error_time_secs = true; if (this_present_error_time_secs || that_present_error_time_secs) { if (!(this_present_error_time_secs && that_present_error_time_secs)) return false; if (this.error_time_secs != that.error_time_secs) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_error = true && (is_set_error()); builder.append(present_error); if (present_error) builder.append(error); boolean present_error_time_secs = true; builder.append(present_error_time_secs); if (present_error_time_secs) builder.append(error_time_secs); return builder.toHashCode(); } public int compareTo(ErrorInfo other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; ErrorInfo typedOther = (ErrorInfo)other; lastComparison = Boolean.valueOf(is_set_error()).compareTo(typedOther.is_set_error()); if (lastComparison != 0) { return lastComparison; } if (is_set_error()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.error, typedOther.error); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_error_time_secs()).compareTo(typedOther.is_set_error_time_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_error_time_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.error_time_secs, typedOther.error_time_secs); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ERROR if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.error = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // ERROR_TIME_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.error_time_secs = iprot.readI32(); set_error_time_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.error != null) { oprot.writeFieldBegin(ERROR_FIELD_DESC); oprot.writeString(this.error); oprot.writeFieldEnd(); } oprot.writeFieldBegin(ERROR_TIME_SECS_FIELD_DESC); oprot.writeI32(this.error_time_secs); oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("ErrorInfo("); boolean first = true; sb.append("error:"); if (this.error == null) { sb.append("null"); } else { sb.append(this.error); } first = false; if (!first) sb.append(", "); sb.append("error_time_secs:"); sb.append(this.error_time_secs); first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_error()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'error' is unset! Struct:" + toString()); } if (!is_set_error_time_secs()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'error_time_secs' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ExecutorInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ExecutorInfo"); private static final org.apache.thrift7.protocol.TField TASK_START_FIELD_DESC = new org.apache.thrift7.protocol.TField("task_start", org.apache.thrift7.protocol.TType.I32, (short)1); private static final org.apache.thrift7.protocol.TField TASK_END_FIELD_DESC = new org.apache.thrift7.protocol.TField("task_end", org.apache.thrift7.protocol.TType.I32, (short)2); private int task_start; // required private int task_end; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { TASK_START((short)1, "task_start"), TASK_END((short)2, "task_end"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // TASK_START return TASK_START; case 2: // TASK_END return TASK_END; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __TASK_START_ISSET_ID = 0; private static final int __TASK_END_ISSET_ID = 1; private BitSet __isset_bit_vector = new BitSet(2); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.TASK_START, new org.apache.thrift7.meta_data.FieldMetaData("task_start", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.TASK_END, new org.apache.thrift7.meta_data.FieldMetaData("task_end", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorInfo.class, metaDataMap); } public ExecutorInfo() { } public ExecutorInfo( int task_start, int task_end) { this(); this.task_start = task_start; set_task_start_isSet(true); this.task_end = task_end; set_task_end_isSet(true); } /** * Performs a deep copy on other. */ public ExecutorInfo(ExecutorInfo other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); this.task_start = other.task_start; this.task_end = other.task_end; } public ExecutorInfo deepCopy() { return new ExecutorInfo(this); } @Override public void clear() { set_task_start_isSet(false); this.task_start = 0; set_task_end_isSet(false); this.task_end = 0; } public int get_task_start() { return this.task_start; } public void set_task_start(int task_start) { this.task_start = task_start; set_task_start_isSet(true); } public void unset_task_start() { __isset_bit_vector.clear(__TASK_START_ISSET_ID); } /** Returns true if field task_start is set (has been assigned a value) and false otherwise */ public boolean is_set_task_start() { return __isset_bit_vector.get(__TASK_START_ISSET_ID); } public void set_task_start_isSet(boolean value) { __isset_bit_vector.set(__TASK_START_ISSET_ID, value); } public int get_task_end() { return this.task_end; } public void set_task_end(int task_end) { this.task_end = task_end; set_task_end_isSet(true); } public void unset_task_end() { __isset_bit_vector.clear(__TASK_END_ISSET_ID); } /** Returns true if field task_end is set (has been assigned a value) and false otherwise */ public boolean is_set_task_end() { return __isset_bit_vector.get(__TASK_END_ISSET_ID); } public void set_task_end_isSet(boolean value) { __isset_bit_vector.set(__TASK_END_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { switch (field) { case TASK_START: if (value == null) { unset_task_start(); } else { set_task_start((Integer)value); } break; case TASK_END: if (value == null) { unset_task_end(); } else { set_task_end((Integer)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case TASK_START: return Integer.valueOf(get_task_start()); case TASK_END: return Integer.valueOf(get_task_end()); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case TASK_START: return is_set_task_start(); case TASK_END: return is_set_task_end(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof ExecutorInfo) return this.equals((ExecutorInfo)that); return false; } public boolean equals(ExecutorInfo that) { if (that == null) return false; boolean this_present_task_start = true; boolean that_present_task_start = true; if (this_present_task_start || that_present_task_start) { if (!(this_present_task_start && that_present_task_start)) return false; if (this.task_start != that.task_start) return false; } boolean this_present_task_end = true; boolean that_present_task_end = true; if (this_present_task_end || that_present_task_end) { if (!(this_present_task_end && that_present_task_end)) return false; if (this.task_end != that.task_end) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_task_start = true; builder.append(present_task_start); if (present_task_start) builder.append(task_start); boolean present_task_end = true; builder.append(present_task_end); if (present_task_end) builder.append(task_end); return builder.toHashCode(); } public int compareTo(ExecutorInfo other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; ExecutorInfo typedOther = (ExecutorInfo)other; lastComparison = Boolean.valueOf(is_set_task_start()).compareTo(typedOther.is_set_task_start()); if (lastComparison != 0) { return lastComparison; } if (is_set_task_start()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.task_start, typedOther.task_start); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_task_end()).compareTo(typedOther.is_set_task_end()); if (lastComparison != 0) { return lastComparison; } if (is_set_task_end()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.task_end, typedOther.task_end); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // TASK_START if (field.type == org.apache.thrift7.protocol.TType.I32) { this.task_start = iprot.readI32(); set_task_start_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // TASK_END if (field.type == org.apache.thrift7.protocol.TType.I32) { this.task_end = iprot.readI32(); set_task_end_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldBegin(TASK_START_FIELD_DESC); oprot.writeI32(this.task_start); oprot.writeFieldEnd(); oprot.writeFieldBegin(TASK_END_FIELD_DESC); oprot.writeI32(this.task_end); oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("ExecutorInfo("); boolean first = true; sb.append("task_start:"); sb.append(this.task_start); first = false; if (!first) sb.append(", "); sb.append("task_end:"); sb.append(this.task_end); first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_task_start()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'task_start' is unset! Struct:" + toString()); } if (!is_set_task_end()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'task_end' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorSpecificStats.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ExecutorSpecificStats extends org.apache.thrift7.TUnion { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ExecutorSpecificStats"); private static final org.apache.thrift7.protocol.TField BOLT_FIELD_DESC = new org.apache.thrift7.protocol.TField("bolt", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField SPOUT_FIELD_DESC = new org.apache.thrift7.protocol.TField("spout", org.apache.thrift7.protocol.TType.STRUCT, (short)2); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { BOLT((short)1, "bolt"), SPOUT((short)2, "spout"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // BOLT return BOLT; case 2: // SPOUT return SPOUT; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.BOLT, new org.apache.thrift7.meta_data.FieldMetaData("bolt", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, BoltStats.class))); tmpMap.put(_Fields.SPOUT, new org.apache.thrift7.meta_data.FieldMetaData("spout", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SpoutStats.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorSpecificStats.class, metaDataMap); } public ExecutorSpecificStats() { super(); } public ExecutorSpecificStats(_Fields setField, Object value) { super(setField, value); } public ExecutorSpecificStats(ExecutorSpecificStats other) { super(other); } public ExecutorSpecificStats deepCopy() { return new ExecutorSpecificStats(this); } public static ExecutorSpecificStats bolt(BoltStats value) { ExecutorSpecificStats x = new ExecutorSpecificStats(); x.set_bolt(value); return x; } public static ExecutorSpecificStats spout(SpoutStats value) { ExecutorSpecificStats x = new ExecutorSpecificStats(); x.set_spout(value); return x; } @Override protected void checkType(_Fields setField, Object value) throws ClassCastException { switch (setField) { case BOLT: if (value instanceof BoltStats) { break; } throw new ClassCastException("Was expecting value of type BoltStats for field 'bolt', but got " + value.getClass().getSimpleName()); case SPOUT: if (value instanceof SpoutStats) { break; } throw new ClassCastException("Was expecting value of type SpoutStats for field 'spout', but got " + value.getClass().getSimpleName()); default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { _Fields setField = _Fields.findByThriftId(field.id); if (setField != null) { switch (setField) { case BOLT: if (field.type == BOLT_FIELD_DESC.type) { BoltStats bolt; bolt = new BoltStats(); bolt.read(iprot); return bolt; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case SPOUT: if (field.type == SPOUT_FIELD_DESC.type) { SpoutStats spout; spout = new SpoutStats(); spout.read(iprot); return spout; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } default: throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } } @Override protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { switch (setField_) { case BOLT: BoltStats bolt = (BoltStats)value_; bolt.write(oprot); return; case SPOUT: SpoutStats spout = (SpoutStats)value_; spout.write(oprot); return; default: throw new IllegalStateException("Cannot write union with unknown field " + setField_); } } @Override protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { switch (setField) { case BOLT: return BOLT_FIELD_DESC; case SPOUT: return SPOUT_FIELD_DESC; default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected org.apache.thrift7.protocol.TStruct getStructDesc() { return STRUCT_DESC; } @Override protected _Fields enumForId(short id) { return _Fields.findByThriftIdOrThrow(id); } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public BoltStats get_bolt() { if (getSetField() == _Fields.BOLT) { return (BoltStats)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'bolt' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_bolt(BoltStats value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.BOLT; value_ = value; } public SpoutStats get_spout() { if (getSetField() == _Fields.SPOUT) { return (SpoutStats)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'spout' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_spout(SpoutStats value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.SPOUT; value_ = value; } public boolean is_set_bolt() { return setField_ == _Fields.BOLT; } public boolean is_set_spout() { return setField_ == _Fields.SPOUT; } public boolean equals(Object other) { if (other instanceof ExecutorSpecificStats) { return equals((ExecutorSpecificStats)other); } else { return false; } } public boolean equals(ExecutorSpecificStats other) { return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); } @Override public int compareTo(ExecutorSpecificStats other) { int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); if (lastComparison == 0) { return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); } return lastComparison; } @Override public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(); hcb.append(this.getClass().getName()); org.apache.thrift7.TFieldIdEnum setField = getSetField(); if (setField != null) { hcb.append(setField.getThriftFieldId()); Object value = getFieldValue(); if (value instanceof org.apache.thrift7.TEnum) { hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); } else { hcb.append(value); } } return hcb.toHashCode(); } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ExecutorStats implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ExecutorStats"); private static final org.apache.thrift7.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift7.protocol.TField("emitted", org.apache.thrift7.protocol.TType.MAP, (short)1); private static final org.apache.thrift7.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift7.protocol.TField("transferred", org.apache.thrift7.protocol.TType.MAP, (short)2); private static final org.apache.thrift7.protocol.TField SPECIFIC_FIELD_DESC = new org.apache.thrift7.protocol.TField("specific", org.apache.thrift7.protocol.TType.STRUCT, (short)3); private Map> emitted; // required private Map> transferred; // required private ExecutorSpecificStats specific; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { EMITTED((short)1, "emitted"), TRANSFERRED((short)2, "transferred"), SPECIFIC((short)3, "specific"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // EMITTED return EMITTED; case 2: // TRANSFERRED return TRANSFERRED; case 3: // SPECIFIC return SPECIFIC; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.EMITTED, new org.apache.thrift7.meta_data.FieldMetaData("emitted", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift7.meta_data.FieldMetaData("transferred", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); tmpMap.put(_Fields.SPECIFIC, new org.apache.thrift7.meta_data.FieldMetaData("specific", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ExecutorSpecificStats.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorStats.class, metaDataMap); } public ExecutorStats() { } public ExecutorStats( Map> emitted, Map> transferred, ExecutorSpecificStats specific) { this(); this.emitted = emitted; this.transferred = transferred; this.specific = specific; } /** * Performs a deep copy on other. */ public ExecutorStats(ExecutorStats other) { if (other.is_set_emitted()) { Map> __this__emitted = new HashMap>(); for (Map.Entry> other_element : other.emitted.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__emitted_copy_key = other_element_key; Map __this__emitted_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { String other_element_value_element_key = other_element_value_element.getKey(); Long other_element_value_element_value = other_element_value_element.getValue(); String __this__emitted_copy_value_copy_key = other_element_value_element_key; Long __this__emitted_copy_value_copy_value = other_element_value_element_value; __this__emitted_copy_value.put(__this__emitted_copy_value_copy_key, __this__emitted_copy_value_copy_value); } __this__emitted.put(__this__emitted_copy_key, __this__emitted_copy_value); } this.emitted = __this__emitted; } if (other.is_set_transferred()) { Map> __this__transferred = new HashMap>(); for (Map.Entry> other_element : other.transferred.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__transferred_copy_key = other_element_key; Map __this__transferred_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { String other_element_value_element_key = other_element_value_element.getKey(); Long other_element_value_element_value = other_element_value_element.getValue(); String __this__transferred_copy_value_copy_key = other_element_value_element_key; Long __this__transferred_copy_value_copy_value = other_element_value_element_value; __this__transferred_copy_value.put(__this__transferred_copy_value_copy_key, __this__transferred_copy_value_copy_value); } __this__transferred.put(__this__transferred_copy_key, __this__transferred_copy_value); } this.transferred = __this__transferred; } if (other.is_set_specific()) { this.specific = new ExecutorSpecificStats(other.specific); } } public ExecutorStats deepCopy() { return new ExecutorStats(this); } @Override public void clear() { this.emitted = null; this.transferred = null; this.specific = null; } public int get_emitted_size() { return (this.emitted == null) ? 0 : this.emitted.size(); } public void put_to_emitted(String key, Map val) { if (this.emitted == null) { this.emitted = new HashMap>(); } this.emitted.put(key, val); } public Map> get_emitted() { return this.emitted; } public void set_emitted(Map> emitted) { this.emitted = emitted; } public void unset_emitted() { this.emitted = null; } /** Returns true if field emitted is set (has been assigned a value) and false otherwise */ public boolean is_set_emitted() { return this.emitted != null; } public void set_emitted_isSet(boolean value) { if (!value) { this.emitted = null; } } public int get_transferred_size() { return (this.transferred == null) ? 0 : this.transferred.size(); } public void put_to_transferred(String key, Map val) { if (this.transferred == null) { this.transferred = new HashMap>(); } this.transferred.put(key, val); } public Map> get_transferred() { return this.transferred; } public void set_transferred(Map> transferred) { this.transferred = transferred; } public void unset_transferred() { this.transferred = null; } /** Returns true if field transferred is set (has been assigned a value) and false otherwise */ public boolean is_set_transferred() { return this.transferred != null; } public void set_transferred_isSet(boolean value) { if (!value) { this.transferred = null; } } public ExecutorSpecificStats get_specific() { return this.specific; } public void set_specific(ExecutorSpecificStats specific) { this.specific = specific; } public void unset_specific() { this.specific = null; } /** Returns true if field specific is set (has been assigned a value) and false otherwise */ public boolean is_set_specific() { return this.specific != null; } public void set_specific_isSet(boolean value) { if (!value) { this.specific = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case EMITTED: if (value == null) { unset_emitted(); } else { set_emitted((Map>)value); } break; case TRANSFERRED: if (value == null) { unset_transferred(); } else { set_transferred((Map>)value); } break; case SPECIFIC: if (value == null) { unset_specific(); } else { set_specific((ExecutorSpecificStats)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case EMITTED: return get_emitted(); case TRANSFERRED: return get_transferred(); case SPECIFIC: return get_specific(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case EMITTED: return is_set_emitted(); case TRANSFERRED: return is_set_transferred(); case SPECIFIC: return is_set_specific(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof ExecutorStats) return this.equals((ExecutorStats)that); return false; } public boolean equals(ExecutorStats that) { if (that == null) return false; boolean this_present_emitted = true && this.is_set_emitted(); boolean that_present_emitted = true && that.is_set_emitted(); if (this_present_emitted || that_present_emitted) { if (!(this_present_emitted && that_present_emitted)) return false; if (!this.emitted.equals(that.emitted)) return false; } boolean this_present_transferred = true && this.is_set_transferred(); boolean that_present_transferred = true && that.is_set_transferred(); if (this_present_transferred || that_present_transferred) { if (!(this_present_transferred && that_present_transferred)) return false; if (!this.transferred.equals(that.transferred)) return false; } boolean this_present_specific = true && this.is_set_specific(); boolean that_present_specific = true && that.is_set_specific(); if (this_present_specific || that_present_specific) { if (!(this_present_specific && that_present_specific)) return false; if (!this.specific.equals(that.specific)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_emitted = true && (is_set_emitted()); builder.append(present_emitted); if (present_emitted) builder.append(emitted); boolean present_transferred = true && (is_set_transferred()); builder.append(present_transferred); if (present_transferred) builder.append(transferred); boolean present_specific = true && (is_set_specific()); builder.append(present_specific); if (present_specific) builder.append(specific); return builder.toHashCode(); } public int compareTo(ExecutorStats other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; ExecutorStats typedOther = (ExecutorStats)other; lastComparison = Boolean.valueOf(is_set_emitted()).compareTo(typedOther.is_set_emitted()); if (lastComparison != 0) { return lastComparison; } if (is_set_emitted()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.emitted, typedOther.emitted); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_transferred()).compareTo(typedOther.is_set_transferred()); if (lastComparison != 0) { return lastComparison; } if (is_set_transferred()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.transferred, typedOther.transferred); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_specific()).compareTo(typedOther.is_set_specific()); if (lastComparison != 0) { return lastComparison; } if (is_set_specific()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.specific, typedOther.specific); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // EMITTED if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map125 = iprot.readMapBegin(); this.emitted = new HashMap>(2*_map125.size); for (int _i126 = 0; _i126 < _map125.size; ++_i126) { String _key127; // required Map _val128; // required _key127 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map129 = iprot.readMapBegin(); _val128 = new HashMap(2*_map129.size); for (int _i130 = 0; _i130 < _map129.size; ++_i130) { String _key131; // required long _val132; // required _key131 = iprot.readString(); _val132 = iprot.readI64(); _val128.put(_key131, _val132); } iprot.readMapEnd(); } this.emitted.put(_key127, _val128); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // TRANSFERRED if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map133 = iprot.readMapBegin(); this.transferred = new HashMap>(2*_map133.size); for (int _i134 = 0; _i134 < _map133.size; ++_i134) { String _key135; // required Map _val136; // required _key135 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map137 = iprot.readMapBegin(); _val136 = new HashMap(2*_map137.size); for (int _i138 = 0; _i138 < _map137.size; ++_i138) { String _key139; // required long _val140; // required _key139 = iprot.readString(); _val140 = iprot.readI64(); _val136.put(_key139, _val140); } iprot.readMapEnd(); } this.transferred.put(_key135, _val136); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // SPECIFIC if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.specific = new ExecutorSpecificStats(); this.specific.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.emitted != null) { oprot.writeFieldBegin(EMITTED_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.emitted.size())); for (Map.Entry> _iter141 : this.emitted.entrySet()) { oprot.writeString(_iter141.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I64, _iter141.getValue().size())); for (Map.Entry _iter142 : _iter141.getValue().entrySet()) { oprot.writeString(_iter142.getKey()); oprot.writeI64(_iter142.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.transferred != null) { oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.transferred.size())); for (Map.Entry> _iter143 : this.transferred.entrySet()) { oprot.writeString(_iter143.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I64, _iter143.getValue().size())); for (Map.Entry _iter144 : _iter143.getValue().entrySet()) { oprot.writeString(_iter144.getKey()); oprot.writeI64(_iter144.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.specific != null) { oprot.writeFieldBegin(SPECIFIC_FIELD_DESC); this.specific.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("ExecutorStats("); boolean first = true; sb.append("emitted:"); if (this.emitted == null) { sb.append("null"); } else { sb.append(this.emitted); } first = false; if (!first) sb.append(", "); sb.append("transferred:"); if (this.transferred == null) { sb.append("null"); } else { sb.append(this.transferred); } first = false; if (!first) sb.append(", "); sb.append("specific:"); if (this.specific == null) { sb.append("null"); } else { sb.append(this.specific); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_emitted()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'emitted' is unset! Struct:" + toString()); } if (!is_set_transferred()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'transferred' is unset! Struct:" + toString()); } if (!is_set_specific()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'specific' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ExecutorSummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ExecutorSummary"); private static final org.apache.thrift7.protocol.TField EXECUTOR_INFO_FIELD_DESC = new org.apache.thrift7.protocol.TField("executor_info", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("component_id", org.apache.thrift7.protocol.TType.STRING, (short)2); private static final org.apache.thrift7.protocol.TField HOST_FIELD_DESC = new org.apache.thrift7.protocol.TField("host", org.apache.thrift7.protocol.TType.STRING, (short)3); private static final org.apache.thrift7.protocol.TField PORT_FIELD_DESC = new org.apache.thrift7.protocol.TField("port", org.apache.thrift7.protocol.TType.I32, (short)4); private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)5); private static final org.apache.thrift7.protocol.TField STATS_FIELD_DESC = new org.apache.thrift7.protocol.TField("stats", org.apache.thrift7.protocol.TType.STRUCT, (short)7); private ExecutorInfo executor_info; // required private String component_id; // required private String host; // required private int port; // required private int uptime_secs; // required private ExecutorStats stats; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { EXECUTOR_INFO((short)1, "executor_info"), COMPONENT_ID((short)2, "component_id"), HOST((short)3, "host"), PORT((short)4, "port"), UPTIME_SECS((short)5, "uptime_secs"), STATS((short)7, "stats"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // EXECUTOR_INFO return EXECUTOR_INFO; case 2: // COMPONENT_ID return COMPONENT_ID; case 3: // HOST return HOST; case 4: // PORT return PORT; case 5: // UPTIME_SECS return UPTIME_SECS; case 7: // STATS return STATS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __PORT_ISSET_ID = 0; private static final int __UPTIME_SECS_ISSET_ID = 1; private BitSet __isset_bit_vector = new BitSet(2); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.EXECUTOR_INFO, new org.apache.thrift7.meta_data.FieldMetaData("executor_info", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ExecutorInfo.class))); tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift7.meta_data.FieldMetaData("component_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.HOST, new org.apache.thrift7.meta_data.FieldMetaData("host", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.PORT, new org.apache.thrift7.meta_data.FieldMetaData("port", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.STATS, new org.apache.thrift7.meta_data.FieldMetaData("stats", org.apache.thrift7.TFieldRequirementType.OPTIONAL, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ExecutorStats.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorSummary.class, metaDataMap); } public ExecutorSummary() { } public ExecutorSummary( ExecutorInfo executor_info, String component_id, String host, int port, int uptime_secs) { this(); this.executor_info = executor_info; this.component_id = component_id; this.host = host; this.port = port; set_port_isSet(true); this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); } /** * Performs a deep copy on other. */ public ExecutorSummary(ExecutorSummary other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_executor_info()) { this.executor_info = new ExecutorInfo(other.executor_info); } if (other.is_set_component_id()) { this.component_id = other.component_id; } if (other.is_set_host()) { this.host = other.host; } this.port = other.port; this.uptime_secs = other.uptime_secs; if (other.is_set_stats()) { this.stats = new ExecutorStats(other.stats); } } public ExecutorSummary deepCopy() { return new ExecutorSummary(this); } @Override public void clear() { this.executor_info = null; this.component_id = null; this.host = null; set_port_isSet(false); this.port = 0; set_uptime_secs_isSet(false); this.uptime_secs = 0; this.stats = null; } public ExecutorInfo get_executor_info() { return this.executor_info; } public void set_executor_info(ExecutorInfo executor_info) { this.executor_info = executor_info; } public void unset_executor_info() { this.executor_info = null; } /** Returns true if field executor_info is set (has been assigned a value) and false otherwise */ public boolean is_set_executor_info() { return this.executor_info != null; } public void set_executor_info_isSet(boolean value) { if (!value) { this.executor_info = null; } } public String get_component_id() { return this.component_id; } public void set_component_id(String component_id) { this.component_id = component_id; } public void unset_component_id() { this.component_id = null; } /** Returns true if field component_id is set (has been assigned a value) and false otherwise */ public boolean is_set_component_id() { return this.component_id != null; } public void set_component_id_isSet(boolean value) { if (!value) { this.component_id = null; } } public String get_host() { return this.host; } public void set_host(String host) { this.host = host; } public void unset_host() { this.host = null; } /** Returns true if field host is set (has been assigned a value) and false otherwise */ public boolean is_set_host() { return this.host != null; } public void set_host_isSet(boolean value) { if (!value) { this.host = null; } } public int get_port() { return this.port; } public void set_port(int port) { this.port = port; set_port_isSet(true); } public void unset_port() { __isset_bit_vector.clear(__PORT_ISSET_ID); } /** Returns true if field port is set (has been assigned a value) and false otherwise */ public boolean is_set_port() { return __isset_bit_vector.get(__PORT_ISSET_ID); } public void set_port_isSet(boolean value) { __isset_bit_vector.set(__PORT_ISSET_ID, value); } public int get_uptime_secs() { return this.uptime_secs; } public void set_uptime_secs(int uptime_secs) { this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); } public void unset_uptime_secs() { __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); } /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_uptime_secs() { return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); } public void set_uptime_secs_isSet(boolean value) { __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); } public ExecutorStats get_stats() { return this.stats; } public void set_stats(ExecutorStats stats) { this.stats = stats; } public void unset_stats() { this.stats = null; } /** Returns true if field stats is set (has been assigned a value) and false otherwise */ public boolean is_set_stats() { return this.stats != null; } public void set_stats_isSet(boolean value) { if (!value) { this.stats = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case EXECUTOR_INFO: if (value == null) { unset_executor_info(); } else { set_executor_info((ExecutorInfo)value); } break; case COMPONENT_ID: if (value == null) { unset_component_id(); } else { set_component_id((String)value); } break; case HOST: if (value == null) { unset_host(); } else { set_host((String)value); } break; case PORT: if (value == null) { unset_port(); } else { set_port((Integer)value); } break; case UPTIME_SECS: if (value == null) { unset_uptime_secs(); } else { set_uptime_secs((Integer)value); } break; case STATS: if (value == null) { unset_stats(); } else { set_stats((ExecutorStats)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case EXECUTOR_INFO: return get_executor_info(); case COMPONENT_ID: return get_component_id(); case HOST: return get_host(); case PORT: return Integer.valueOf(get_port()); case UPTIME_SECS: return Integer.valueOf(get_uptime_secs()); case STATS: return get_stats(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case EXECUTOR_INFO: return is_set_executor_info(); case COMPONENT_ID: return is_set_component_id(); case HOST: return is_set_host(); case PORT: return is_set_port(); case UPTIME_SECS: return is_set_uptime_secs(); case STATS: return is_set_stats(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof ExecutorSummary) return this.equals((ExecutorSummary)that); return false; } public boolean equals(ExecutorSummary that) { if (that == null) return false; boolean this_present_executor_info = true && this.is_set_executor_info(); boolean that_present_executor_info = true && that.is_set_executor_info(); if (this_present_executor_info || that_present_executor_info) { if (!(this_present_executor_info && that_present_executor_info)) return false; if (!this.executor_info.equals(that.executor_info)) return false; } boolean this_present_component_id = true && this.is_set_component_id(); boolean that_present_component_id = true && that.is_set_component_id(); if (this_present_component_id || that_present_component_id) { if (!(this_present_component_id && that_present_component_id)) return false; if (!this.component_id.equals(that.component_id)) return false; } boolean this_present_host = true && this.is_set_host(); boolean that_present_host = true && that.is_set_host(); if (this_present_host || that_present_host) { if (!(this_present_host && that_present_host)) return false; if (!this.host.equals(that.host)) return false; } boolean this_present_port = true; boolean that_present_port = true; if (this_present_port || that_present_port) { if (!(this_present_port && that_present_port)) return false; if (this.port != that.port) return false; } boolean this_present_uptime_secs = true; boolean that_present_uptime_secs = true; if (this_present_uptime_secs || that_present_uptime_secs) { if (!(this_present_uptime_secs && that_present_uptime_secs)) return false; if (this.uptime_secs != that.uptime_secs) return false; } boolean this_present_stats = true && this.is_set_stats(); boolean that_present_stats = true && that.is_set_stats(); if (this_present_stats || that_present_stats) { if (!(this_present_stats && that_present_stats)) return false; if (!this.stats.equals(that.stats)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_executor_info = true && (is_set_executor_info()); builder.append(present_executor_info); if (present_executor_info) builder.append(executor_info); boolean present_component_id = true && (is_set_component_id()); builder.append(present_component_id); if (present_component_id) builder.append(component_id); boolean present_host = true && (is_set_host()); builder.append(present_host); if (present_host) builder.append(host); boolean present_port = true; builder.append(present_port); if (present_port) builder.append(port); boolean present_uptime_secs = true; builder.append(present_uptime_secs); if (present_uptime_secs) builder.append(uptime_secs); boolean present_stats = true && (is_set_stats()); builder.append(present_stats); if (present_stats) builder.append(stats); return builder.toHashCode(); } public int compareTo(ExecutorSummary other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; ExecutorSummary typedOther = (ExecutorSummary)other; lastComparison = Boolean.valueOf(is_set_executor_info()).compareTo(typedOther.is_set_executor_info()); if (lastComparison != 0) { return lastComparison; } if (is_set_executor_info()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.executor_info, typedOther.executor_info); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(typedOther.is_set_component_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_component_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.component_id, typedOther.component_id); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host()); if (lastComparison != 0) { return lastComparison; } if (is_set_host()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.host, typedOther.host); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port()); if (lastComparison != 0) { return lastComparison; } if (is_set_port()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.port, typedOther.port); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_uptime_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_stats()).compareTo(typedOther.is_set_stats()); if (lastComparison != 0) { return lastComparison; } if (is_set_stats()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.stats, typedOther.stats); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // EXECUTOR_INFO if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.executor_info = new ExecutorInfo(); this.executor_info.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // COMPONENT_ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.component_id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // HOST if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.host = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // PORT if (field.type == org.apache.thrift7.protocol.TType.I32) { this.port = iprot.readI32(); set_port_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 5: // UPTIME_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.uptime_secs = iprot.readI32(); set_uptime_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 7: // STATS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.stats = new ExecutorStats(); this.stats.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.executor_info != null) { oprot.writeFieldBegin(EXECUTOR_INFO_FIELD_DESC); this.executor_info.write(oprot); oprot.writeFieldEnd(); } if (this.component_id != null) { oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); oprot.writeString(this.component_id); oprot.writeFieldEnd(); } if (this.host != null) { oprot.writeFieldBegin(HOST_FIELD_DESC); oprot.writeString(this.host); oprot.writeFieldEnd(); } oprot.writeFieldBegin(PORT_FIELD_DESC); oprot.writeI32(this.port); oprot.writeFieldEnd(); oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); oprot.writeI32(this.uptime_secs); oprot.writeFieldEnd(); if (this.stats != null) { if (is_set_stats()) { oprot.writeFieldBegin(STATS_FIELD_DESC); this.stats.write(oprot); oprot.writeFieldEnd(); } } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("ExecutorSummary("); boolean first = true; sb.append("executor_info:"); if (this.executor_info == null) { sb.append("null"); } else { sb.append(this.executor_info); } first = false; if (!first) sb.append(", "); sb.append("component_id:"); if (this.component_id == null) { sb.append("null"); } else { sb.append(this.component_id); } first = false; if (!first) sb.append(", "); sb.append("host:"); if (this.host == null) { sb.append("null"); } else { sb.append(this.host); } first = false; if (!first) sb.append(", "); sb.append("port:"); sb.append(this.port); first = false; if (!first) sb.append(", "); sb.append("uptime_secs:"); sb.append(this.uptime_secs); first = false; if (is_set_stats()) { if (!first) sb.append(", "); sb.append("stats:"); if (this.stats == null) { sb.append("null"); } else { sb.append(this.stats); } first = false; } sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_executor_info()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'executor_info' is unset! Struct:" + toString()); } if (!is_set_component_id()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'component_id' is unset! Struct:" + toString()); } if (!is_set_host()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString()); } if (!is_set_port()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString()); } if (!is_set_uptime_secs()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class GlobalStreamId implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("GlobalStreamId"); private static final org.apache.thrift7.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("componentId", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField STREAM_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("streamId", org.apache.thrift7.protocol.TType.STRING, (short)2); private String componentId; // required private String streamId; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { COMPONENT_ID((short)1, "componentId"), STREAM_ID((short)2, "streamId"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // COMPONENT_ID return COMPONENT_ID; case 2: // STREAM_ID return STREAM_ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift7.meta_data.FieldMetaData("componentId", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.STREAM_ID, new org.apache.thrift7.meta_data.FieldMetaData("streamId", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(GlobalStreamId.class, metaDataMap); } public GlobalStreamId() { } public GlobalStreamId( String componentId, String streamId) { this(); this.componentId = componentId; this.streamId = streamId; } /** * Performs a deep copy on other. */ public GlobalStreamId(GlobalStreamId other) { if (other.is_set_componentId()) { this.componentId = other.componentId; } if (other.is_set_streamId()) { this.streamId = other.streamId; } } public GlobalStreamId deepCopy() { return new GlobalStreamId(this); } @Override public void clear() { this.componentId = null; this.streamId = null; } public String get_componentId() { return this.componentId; } public void set_componentId(String componentId) { this.componentId = componentId; } public void unset_componentId() { this.componentId = null; } /** Returns true if field componentId is set (has been assigned a value) and false otherwise */ public boolean is_set_componentId() { return this.componentId != null; } public void set_componentId_isSet(boolean value) { if (!value) { this.componentId = null; } } public String get_streamId() { return this.streamId; } public void set_streamId(String streamId) { this.streamId = streamId; } public void unset_streamId() { this.streamId = null; } /** Returns true if field streamId is set (has been assigned a value) and false otherwise */ public boolean is_set_streamId() { return this.streamId != null; } public void set_streamId_isSet(boolean value) { if (!value) { this.streamId = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case COMPONENT_ID: if (value == null) { unset_componentId(); } else { set_componentId((String)value); } break; case STREAM_ID: if (value == null) { unset_streamId(); } else { set_streamId((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case COMPONENT_ID: return get_componentId(); case STREAM_ID: return get_streamId(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case COMPONENT_ID: return is_set_componentId(); case STREAM_ID: return is_set_streamId(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof GlobalStreamId) return this.equals((GlobalStreamId)that); return false; } public boolean equals(GlobalStreamId that) { if (that == null) return false; boolean this_present_componentId = true && this.is_set_componentId(); boolean that_present_componentId = true && that.is_set_componentId(); if (this_present_componentId || that_present_componentId) { if (!(this_present_componentId && that_present_componentId)) return false; if (!this.componentId.equals(that.componentId)) return false; } boolean this_present_streamId = true && this.is_set_streamId(); boolean that_present_streamId = true && that.is_set_streamId(); if (this_present_streamId || that_present_streamId) { if (!(this_present_streamId && that_present_streamId)) return false; if (!this.streamId.equals(that.streamId)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_componentId = true && (is_set_componentId()); builder.append(present_componentId); if (present_componentId) builder.append(componentId); boolean present_streamId = true && (is_set_streamId()); builder.append(present_streamId); if (present_streamId) builder.append(streamId); return builder.toHashCode(); } public int compareTo(GlobalStreamId other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; GlobalStreamId typedOther = (GlobalStreamId)other; lastComparison = Boolean.valueOf(is_set_componentId()).compareTo(typedOther.is_set_componentId()); if (lastComparison != 0) { return lastComparison; } if (is_set_componentId()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.componentId, typedOther.componentId); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_streamId()).compareTo(typedOther.is_set_streamId()); if (lastComparison != 0) { return lastComparison; } if (is_set_streamId()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.streamId, typedOther.streamId); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // COMPONENT_ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.componentId = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // STREAM_ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.streamId = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.componentId != null) { oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); oprot.writeString(this.componentId); oprot.writeFieldEnd(); } if (this.streamId != null) { oprot.writeFieldBegin(STREAM_ID_FIELD_DESC); oprot.writeString(this.streamId); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("GlobalStreamId("); boolean first = true; sb.append("componentId:"); if (this.componentId == null) { sb.append("null"); } else { sb.append(this.componentId); } first = false; if (!first) sb.append(", "); sb.append("streamId:"); if (this.streamId == null) { sb.append("null"); } else { sb.append(this.streamId); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_componentId()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'componentId' is unset! Struct:" + toString()); } if (!is_set_streamId()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'streamId' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/Grouping.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class Grouping extends org.apache.thrift7.TUnion { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("Grouping"); private static final org.apache.thrift7.protocol.TField FIELDS_FIELD_DESC = new org.apache.thrift7.protocol.TField("fields", org.apache.thrift7.protocol.TType.LIST, (short)1); private static final org.apache.thrift7.protocol.TField SHUFFLE_FIELD_DESC = new org.apache.thrift7.protocol.TField("shuffle", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private static final org.apache.thrift7.protocol.TField ALL_FIELD_DESC = new org.apache.thrift7.protocol.TField("all", org.apache.thrift7.protocol.TType.STRUCT, (short)3); private static final org.apache.thrift7.protocol.TField NONE_FIELD_DESC = new org.apache.thrift7.protocol.TField("none", org.apache.thrift7.protocol.TType.STRUCT, (short)4); private static final org.apache.thrift7.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("direct", org.apache.thrift7.protocol.TType.STRUCT, (short)5); private static final org.apache.thrift7.protocol.TField CUSTOM_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("custom_object", org.apache.thrift7.protocol.TType.STRUCT, (short)6); private static final org.apache.thrift7.protocol.TField CUSTOM_SERIALIZED_FIELD_DESC = new org.apache.thrift7.protocol.TField("custom_serialized", org.apache.thrift7.protocol.TType.STRING, (short)7); private static final org.apache.thrift7.protocol.TField LOCAL_OR_SHUFFLE_FIELD_DESC = new org.apache.thrift7.protocol.TField("local_or_shuffle", org.apache.thrift7.protocol.TType.STRUCT, (short)8); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { FIELDS((short)1, "fields"), SHUFFLE((short)2, "shuffle"), ALL((short)3, "all"), NONE((short)4, "none"), DIRECT((short)5, "direct"), CUSTOM_OBJECT((short)6, "custom_object"), CUSTOM_SERIALIZED((short)7, "custom_serialized"), LOCAL_OR_SHUFFLE((short)8, "local_or_shuffle"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // FIELDS return FIELDS; case 2: // SHUFFLE return SHUFFLE; case 3: // ALL return ALL; case 4: // NONE return NONE; case 5: // DIRECT return DIRECT; case 6: // CUSTOM_OBJECT return CUSTOM_OBJECT; case 7: // CUSTOM_SERIALIZED return CUSTOM_SERIALIZED; case 8: // LOCAL_OR_SHUFFLE return LOCAL_OR_SHUFFLE; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.FIELDS, new org.apache.thrift7.meta_data.FieldMetaData("fields", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING)))); tmpMap.put(_Fields.SHUFFLE, new org.apache.thrift7.meta_data.FieldMetaData("shuffle", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); tmpMap.put(_Fields.ALL, new org.apache.thrift7.meta_data.FieldMetaData("all", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); tmpMap.put(_Fields.NONE, new org.apache.thrift7.meta_data.FieldMetaData("none", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); tmpMap.put(_Fields.DIRECT, new org.apache.thrift7.meta_data.FieldMetaData("direct", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); tmpMap.put(_Fields.CUSTOM_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("custom_object", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, JavaObject.class))); tmpMap.put(_Fields.CUSTOM_SERIALIZED, new org.apache.thrift7.meta_data.FieldMetaData("custom_serialized", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); tmpMap.put(_Fields.LOCAL_OR_SHUFFLE, new org.apache.thrift7.meta_data.FieldMetaData("local_or_shuffle", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(Grouping.class, metaDataMap); } public Grouping() { super(); } public Grouping(_Fields setField, Object value) { super(setField, value); } public Grouping(Grouping other) { super(other); } public Grouping deepCopy() { return new Grouping(this); } public static Grouping fields(List value) { Grouping x = new Grouping(); x.set_fields(value); return x; } public static Grouping shuffle(NullStruct value) { Grouping x = new Grouping(); x.set_shuffle(value); return x; } public static Grouping all(NullStruct value) { Grouping x = new Grouping(); x.set_all(value); return x; } public static Grouping none(NullStruct value) { Grouping x = new Grouping(); x.set_none(value); return x; } public static Grouping direct(NullStruct value) { Grouping x = new Grouping(); x.set_direct(value); return x; } public static Grouping custom_object(JavaObject value) { Grouping x = new Grouping(); x.set_custom_object(value); return x; } public static Grouping custom_serialized(ByteBuffer value) { Grouping x = new Grouping(); x.set_custom_serialized(value); return x; } public static Grouping custom_serialized(byte[] value) { Grouping x = new Grouping(); x.set_custom_serialized(ByteBuffer.wrap(value)); return x; } public static Grouping local_or_shuffle(NullStruct value) { Grouping x = new Grouping(); x.set_local_or_shuffle(value); return x; } @Override protected void checkType(_Fields setField, Object value) throws ClassCastException { switch (setField) { case FIELDS: if (value instanceof List) { break; } throw new ClassCastException("Was expecting value of type List for field 'fields', but got " + value.getClass().getSimpleName()); case SHUFFLE: if (value instanceof NullStruct) { break; } throw new ClassCastException("Was expecting value of type NullStruct for field 'shuffle', but got " + value.getClass().getSimpleName()); case ALL: if (value instanceof NullStruct) { break; } throw new ClassCastException("Was expecting value of type NullStruct for field 'all', but got " + value.getClass().getSimpleName()); case NONE: if (value instanceof NullStruct) { break; } throw new ClassCastException("Was expecting value of type NullStruct for field 'none', but got " + value.getClass().getSimpleName()); case DIRECT: if (value instanceof NullStruct) { break; } throw new ClassCastException("Was expecting value of type NullStruct for field 'direct', but got " + value.getClass().getSimpleName()); case CUSTOM_OBJECT: if (value instanceof JavaObject) { break; } throw new ClassCastException("Was expecting value of type JavaObject for field 'custom_object', but got " + value.getClass().getSimpleName()); case CUSTOM_SERIALIZED: if (value instanceof ByteBuffer) { break; } throw new ClassCastException("Was expecting value of type ByteBuffer for field 'custom_serialized', but got " + value.getClass().getSimpleName()); case LOCAL_OR_SHUFFLE: if (value instanceof NullStruct) { break; } throw new ClassCastException("Was expecting value of type NullStruct for field 'local_or_shuffle', but got " + value.getClass().getSimpleName()); default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { _Fields setField = _Fields.findByThriftId(field.id); if (setField != null) { switch (setField) { case FIELDS: if (field.type == FIELDS_FIELD_DESC.type) { List fields; { org.apache.thrift7.protocol.TList _list4 = iprot.readListBegin(); fields = new ArrayList(_list4.size); for (int _i5 = 0; _i5 < _list4.size; ++_i5) { String _elem6; // required _elem6 = iprot.readString(); fields.add(_elem6); } iprot.readListEnd(); } return fields; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case SHUFFLE: if (field.type == SHUFFLE_FIELD_DESC.type) { NullStruct shuffle; shuffle = new NullStruct(); shuffle.read(iprot); return shuffle; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case ALL: if (field.type == ALL_FIELD_DESC.type) { NullStruct all; all = new NullStruct(); all.read(iprot); return all; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case NONE: if (field.type == NONE_FIELD_DESC.type) { NullStruct none; none = new NullStruct(); none.read(iprot); return none; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case DIRECT: if (field.type == DIRECT_FIELD_DESC.type) { NullStruct direct; direct = new NullStruct(); direct.read(iprot); return direct; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case CUSTOM_OBJECT: if (field.type == CUSTOM_OBJECT_FIELD_DESC.type) { JavaObject custom_object; custom_object = new JavaObject(); custom_object.read(iprot); return custom_object; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case CUSTOM_SERIALIZED: if (field.type == CUSTOM_SERIALIZED_FIELD_DESC.type) { ByteBuffer custom_serialized; custom_serialized = iprot.readBinary(); return custom_serialized; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case LOCAL_OR_SHUFFLE: if (field.type == LOCAL_OR_SHUFFLE_FIELD_DESC.type) { NullStruct local_or_shuffle; local_or_shuffle = new NullStruct(); local_or_shuffle.read(iprot); return local_or_shuffle; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } default: throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } } @Override protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { switch (setField_) { case FIELDS: List fields = (List)value_; { oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRING, fields.size())); for (String _iter7 : fields) { oprot.writeString(_iter7); } oprot.writeListEnd(); } return; case SHUFFLE: NullStruct shuffle = (NullStruct)value_; shuffle.write(oprot); return; case ALL: NullStruct all = (NullStruct)value_; all.write(oprot); return; case NONE: NullStruct none = (NullStruct)value_; none.write(oprot); return; case DIRECT: NullStruct direct = (NullStruct)value_; direct.write(oprot); return; case CUSTOM_OBJECT: JavaObject custom_object = (JavaObject)value_; custom_object.write(oprot); return; case CUSTOM_SERIALIZED: ByteBuffer custom_serialized = (ByteBuffer)value_; oprot.writeBinary(custom_serialized); return; case LOCAL_OR_SHUFFLE: NullStruct local_or_shuffle = (NullStruct)value_; local_or_shuffle.write(oprot); return; default: throw new IllegalStateException("Cannot write union with unknown field " + setField_); } } @Override protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { switch (setField) { case FIELDS: return FIELDS_FIELD_DESC; case SHUFFLE: return SHUFFLE_FIELD_DESC; case ALL: return ALL_FIELD_DESC; case NONE: return NONE_FIELD_DESC; case DIRECT: return DIRECT_FIELD_DESC; case CUSTOM_OBJECT: return CUSTOM_OBJECT_FIELD_DESC; case CUSTOM_SERIALIZED: return CUSTOM_SERIALIZED_FIELD_DESC; case LOCAL_OR_SHUFFLE: return LOCAL_OR_SHUFFLE_FIELD_DESC; default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected org.apache.thrift7.protocol.TStruct getStructDesc() { return STRUCT_DESC; } @Override protected _Fields enumForId(short id) { return _Fields.findByThriftIdOrThrow(id); } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public List get_fields() { if (getSetField() == _Fields.FIELDS) { return (List)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'fields' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_fields(List value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.FIELDS; value_ = value; } public NullStruct get_shuffle() { if (getSetField() == _Fields.SHUFFLE) { return (NullStruct)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'shuffle' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_shuffle(NullStruct value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.SHUFFLE; value_ = value; } public NullStruct get_all() { if (getSetField() == _Fields.ALL) { return (NullStruct)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'all' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_all(NullStruct value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.ALL; value_ = value; } public NullStruct get_none() { if (getSetField() == _Fields.NONE) { return (NullStruct)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'none' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_none(NullStruct value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.NONE; value_ = value; } public NullStruct get_direct() { if (getSetField() == _Fields.DIRECT) { return (NullStruct)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'direct' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_direct(NullStruct value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.DIRECT; value_ = value; } public JavaObject get_custom_object() { if (getSetField() == _Fields.CUSTOM_OBJECT) { return (JavaObject)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'custom_object' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_custom_object(JavaObject value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.CUSTOM_OBJECT; value_ = value; } public byte[] get_custom_serialized() { set_custom_serialized(org.apache.thrift7.TBaseHelper.rightSize(buffer_for_custom_serialized())); ByteBuffer b = buffer_for_custom_serialized(); return b == null ? null : b.array(); } public ByteBuffer buffer_for_custom_serialized() { if (getSetField() == _Fields.CUSTOM_SERIALIZED) { return (ByteBuffer)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'custom_serialized' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_custom_serialized(byte[] value) { set_custom_serialized(ByteBuffer.wrap(value)); } public void set_custom_serialized(ByteBuffer value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.CUSTOM_SERIALIZED; value_ = value; } public NullStruct get_local_or_shuffle() { if (getSetField() == _Fields.LOCAL_OR_SHUFFLE) { return (NullStruct)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'local_or_shuffle' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_local_or_shuffle(NullStruct value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.LOCAL_OR_SHUFFLE; value_ = value; } public boolean is_set_fields() { return setField_ == _Fields.FIELDS; } public boolean is_set_shuffle() { return setField_ == _Fields.SHUFFLE; } public boolean is_set_all() { return setField_ == _Fields.ALL; } public boolean is_set_none() { return setField_ == _Fields.NONE; } public boolean is_set_direct() { return setField_ == _Fields.DIRECT; } public boolean is_set_custom_object() { return setField_ == _Fields.CUSTOM_OBJECT; } public boolean is_set_custom_serialized() { return setField_ == _Fields.CUSTOM_SERIALIZED; } public boolean is_set_local_or_shuffle() { return setField_ == _Fields.LOCAL_OR_SHUFFLE; } public boolean equals(Object other) { if (other instanceof Grouping) { return equals((Grouping)other); } else { return false; } } public boolean equals(Grouping other) { return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); } @Override public int compareTo(Grouping other) { int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); if (lastComparison == 0) { return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); } return lastComparison; } @Override public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(); hcb.append(this.getClass().getName()); org.apache.thrift7.TFieldIdEnum setField = getSetField(); if (setField != null) { hcb.append(setField.getThriftFieldId()); Object value = getFieldValue(); if (value instanceof org.apache.thrift7.TEnum) { hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); } else { hcb.append(value); } } return hcb.toHashCode(); } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class InvalidTopologyException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("InvalidTopologyException"); private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); private String msg; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { MSG((short)1, "msg"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // MSG return MSG; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(InvalidTopologyException.class, metaDataMap); } public InvalidTopologyException() { } public InvalidTopologyException( String msg) { this(); this.msg = msg; } /** * Performs a deep copy on other. */ public InvalidTopologyException(InvalidTopologyException other) { if (other.is_set_msg()) { this.msg = other.msg; } } public InvalidTopologyException deepCopy() { return new InvalidTopologyException(this); } @Override public void clear() { this.msg = null; } public String get_msg() { return this.msg; } public void set_msg(String msg) { this.msg = msg; } public void unset_msg() { this.msg = null; } /** Returns true if field msg is set (has been assigned a value) and false otherwise */ public boolean is_set_msg() { return this.msg != null; } public void set_msg_isSet(boolean value) { if (!value) { this.msg = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case MSG: if (value == null) { unset_msg(); } else { set_msg((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case MSG: return get_msg(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case MSG: return is_set_msg(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof InvalidTopologyException) return this.equals((InvalidTopologyException)that); return false; } public boolean equals(InvalidTopologyException that) { if (that == null) return false; boolean this_present_msg = true && this.is_set_msg(); boolean that_present_msg = true && that.is_set_msg(); if (this_present_msg || that_present_msg) { if (!(this_present_msg && that_present_msg)) return false; if (!this.msg.equals(that.msg)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_msg = true && (is_set_msg()); builder.append(present_msg); if (present_msg) builder.append(msg); return builder.toHashCode(); } public int compareTo(InvalidTopologyException other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; InvalidTopologyException typedOther = (InvalidTopologyException)other; lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); if (lastComparison != 0) { return lastComparison; } if (is_set_msg()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // MSG if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.msg = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.msg != null) { oprot.writeFieldBegin(MSG_FIELD_DESC); oprot.writeString(this.msg); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("InvalidTopologyException("); boolean first = true; sb.append("msg:"); if (this.msg == null) { sb.append("null"); } else { sb.append(this.msg); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_msg()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/JavaObject.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class JavaObject implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("JavaObject"); private static final org.apache.thrift7.protocol.TField FULL_CLASS_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("full_class_name", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField ARGS_LIST_FIELD_DESC = new org.apache.thrift7.protocol.TField("args_list", org.apache.thrift7.protocol.TType.LIST, (short)2); private String full_class_name; // required private List args_list; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { FULL_CLASS_NAME((short)1, "full_class_name"), ARGS_LIST((short)2, "args_list"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // FULL_CLASS_NAME return FULL_CLASS_NAME; case 2: // ARGS_LIST return ARGS_LIST; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.FULL_CLASS_NAME, new org.apache.thrift7.meta_data.FieldMetaData("full_class_name", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.ARGS_LIST, new org.apache.thrift7.meta_data.FieldMetaData("args_list", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, JavaObjectArg.class)))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(JavaObject.class, metaDataMap); } public JavaObject() { } public JavaObject( String full_class_name, List args_list) { this(); this.full_class_name = full_class_name; this.args_list = args_list; } /** * Performs a deep copy on other. */ public JavaObject(JavaObject other) { if (other.is_set_full_class_name()) { this.full_class_name = other.full_class_name; } if (other.is_set_args_list()) { List __this__args_list = new ArrayList(); for (JavaObjectArg other_element : other.args_list) { __this__args_list.add(new JavaObjectArg(other_element)); } this.args_list = __this__args_list; } } public JavaObject deepCopy() { return new JavaObject(this); } @Override public void clear() { this.full_class_name = null; this.args_list = null; } public String get_full_class_name() { return this.full_class_name; } public void set_full_class_name(String full_class_name) { this.full_class_name = full_class_name; } public void unset_full_class_name() { this.full_class_name = null; } /** Returns true if field full_class_name is set (has been assigned a value) and false otherwise */ public boolean is_set_full_class_name() { return this.full_class_name != null; } public void set_full_class_name_isSet(boolean value) { if (!value) { this.full_class_name = null; } } public int get_args_list_size() { return (this.args_list == null) ? 0 : this.args_list.size(); } public java.util.Iterator get_args_list_iterator() { return (this.args_list == null) ? null : this.args_list.iterator(); } public void add_to_args_list(JavaObjectArg elem) { if (this.args_list == null) { this.args_list = new ArrayList(); } this.args_list.add(elem); } public List get_args_list() { return this.args_list; } public void set_args_list(List args_list) { this.args_list = args_list; } public void unset_args_list() { this.args_list = null; } /** Returns true if field args_list is set (has been assigned a value) and false otherwise */ public boolean is_set_args_list() { return this.args_list != null; } public void set_args_list_isSet(boolean value) { if (!value) { this.args_list = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case FULL_CLASS_NAME: if (value == null) { unset_full_class_name(); } else { set_full_class_name((String)value); } break; case ARGS_LIST: if (value == null) { unset_args_list(); } else { set_args_list((List)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case FULL_CLASS_NAME: return get_full_class_name(); case ARGS_LIST: return get_args_list(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case FULL_CLASS_NAME: return is_set_full_class_name(); case ARGS_LIST: return is_set_args_list(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof JavaObject) return this.equals((JavaObject)that); return false; } public boolean equals(JavaObject that) { if (that == null) return false; boolean this_present_full_class_name = true && this.is_set_full_class_name(); boolean that_present_full_class_name = true && that.is_set_full_class_name(); if (this_present_full_class_name || that_present_full_class_name) { if (!(this_present_full_class_name && that_present_full_class_name)) return false; if (!this.full_class_name.equals(that.full_class_name)) return false; } boolean this_present_args_list = true && this.is_set_args_list(); boolean that_present_args_list = true && that.is_set_args_list(); if (this_present_args_list || that_present_args_list) { if (!(this_present_args_list && that_present_args_list)) return false; if (!this.args_list.equals(that.args_list)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_full_class_name = true && (is_set_full_class_name()); builder.append(present_full_class_name); if (present_full_class_name) builder.append(full_class_name); boolean present_args_list = true && (is_set_args_list()); builder.append(present_args_list); if (present_args_list) builder.append(args_list); return builder.toHashCode(); } public int compareTo(JavaObject other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; JavaObject typedOther = (JavaObject)other; lastComparison = Boolean.valueOf(is_set_full_class_name()).compareTo(typedOther.is_set_full_class_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_full_class_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.full_class_name, typedOther.full_class_name); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_args_list()).compareTo(typedOther.is_set_args_list()); if (lastComparison != 0) { return lastComparison; } if (is_set_args_list()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.args_list, typedOther.args_list); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // FULL_CLASS_NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.full_class_name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // ARGS_LIST if (field.type == org.apache.thrift7.protocol.TType.LIST) { { org.apache.thrift7.protocol.TList _list0 = iprot.readListBegin(); this.args_list = new ArrayList(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { JavaObjectArg _elem2; // required _elem2 = new JavaObjectArg(); _elem2.read(iprot); this.args_list.add(_elem2); } iprot.readListEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.full_class_name != null) { oprot.writeFieldBegin(FULL_CLASS_NAME_FIELD_DESC); oprot.writeString(this.full_class_name); oprot.writeFieldEnd(); } if (this.args_list != null) { oprot.writeFieldBegin(ARGS_LIST_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.args_list.size())); for (JavaObjectArg _iter3 : this.args_list) { _iter3.write(oprot); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("JavaObject("); boolean first = true; sb.append("full_class_name:"); if (this.full_class_name == null) { sb.append("null"); } else { sb.append(this.full_class_name); } first = false; if (!first) sb.append(", "); sb.append("args_list:"); if (this.args_list == null) { sb.append("null"); } else { sb.append(this.args_list); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_full_class_name()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'full_class_name' is unset! Struct:" + toString()); } if (!is_set_args_list()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'args_list' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/JavaObjectArg.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class JavaObjectArg extends org.apache.thrift7.TUnion { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("JavaObjectArg"); private static final org.apache.thrift7.protocol.TField INT_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("int_arg", org.apache.thrift7.protocol.TType.I32, (short)1); private static final org.apache.thrift7.protocol.TField LONG_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("long_arg", org.apache.thrift7.protocol.TType.I64, (short)2); private static final org.apache.thrift7.protocol.TField STRING_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("string_arg", org.apache.thrift7.protocol.TType.STRING, (short)3); private static final org.apache.thrift7.protocol.TField BOOL_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("bool_arg", org.apache.thrift7.protocol.TType.BOOL, (short)4); private static final org.apache.thrift7.protocol.TField BINARY_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("binary_arg", org.apache.thrift7.protocol.TType.STRING, (short)5); private static final org.apache.thrift7.protocol.TField DOUBLE_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("double_arg", org.apache.thrift7.protocol.TType.DOUBLE, (short)6); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { INT_ARG((short)1, "int_arg"), LONG_ARG((short)2, "long_arg"), STRING_ARG((short)3, "string_arg"), BOOL_ARG((short)4, "bool_arg"), BINARY_ARG((short)5, "binary_arg"), DOUBLE_ARG((short)6, "double_arg"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // INT_ARG return INT_ARG; case 2: // LONG_ARG return LONG_ARG; case 3: // STRING_ARG return STRING_ARG; case 4: // BOOL_ARG return BOOL_ARG; case 5: // BINARY_ARG return BINARY_ARG; case 6: // DOUBLE_ARG return DOUBLE_ARG; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.INT_ARG, new org.apache.thrift7.meta_data.FieldMetaData("int_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.LONG_ARG, new org.apache.thrift7.meta_data.FieldMetaData("long_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))); tmpMap.put(_Fields.STRING_ARG, new org.apache.thrift7.meta_data.FieldMetaData("string_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.BOOL_ARG, new org.apache.thrift7.meta_data.FieldMetaData("bool_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.BOOL))); tmpMap.put(_Fields.BINARY_ARG, new org.apache.thrift7.meta_data.FieldMetaData("binary_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); tmpMap.put(_Fields.DOUBLE_ARG, new org.apache.thrift7.meta_data.FieldMetaData("double_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(JavaObjectArg.class, metaDataMap); } public JavaObjectArg() { super(); } public JavaObjectArg(_Fields setField, Object value) { super(setField, value); } public JavaObjectArg(JavaObjectArg other) { super(other); } public JavaObjectArg deepCopy() { return new JavaObjectArg(this); } public static JavaObjectArg int_arg(int value) { JavaObjectArg x = new JavaObjectArg(); x.set_int_arg(value); return x; } public static JavaObjectArg long_arg(long value) { JavaObjectArg x = new JavaObjectArg(); x.set_long_arg(value); return x; } public static JavaObjectArg string_arg(String value) { JavaObjectArg x = new JavaObjectArg(); x.set_string_arg(value); return x; } public static JavaObjectArg bool_arg(boolean value) { JavaObjectArg x = new JavaObjectArg(); x.set_bool_arg(value); return x; } public static JavaObjectArg binary_arg(ByteBuffer value) { JavaObjectArg x = new JavaObjectArg(); x.set_binary_arg(value); return x; } public static JavaObjectArg binary_arg(byte[] value) { JavaObjectArg x = new JavaObjectArg(); x.set_binary_arg(ByteBuffer.wrap(value)); return x; } public static JavaObjectArg double_arg(double value) { JavaObjectArg x = new JavaObjectArg(); x.set_double_arg(value); return x; } @Override protected void checkType(_Fields setField, Object value) throws ClassCastException { switch (setField) { case INT_ARG: if (value instanceof Integer) { break; } throw new ClassCastException("Was expecting value of type Integer for field 'int_arg', but got " + value.getClass().getSimpleName()); case LONG_ARG: if (value instanceof Long) { break; } throw new ClassCastException("Was expecting value of type Long for field 'long_arg', but got " + value.getClass().getSimpleName()); case STRING_ARG: if (value instanceof String) { break; } throw new ClassCastException("Was expecting value of type String for field 'string_arg', but got " + value.getClass().getSimpleName()); case BOOL_ARG: if (value instanceof Boolean) { break; } throw new ClassCastException("Was expecting value of type Boolean for field 'bool_arg', but got " + value.getClass().getSimpleName()); case BINARY_ARG: if (value instanceof ByteBuffer) { break; } throw new ClassCastException("Was expecting value of type ByteBuffer for field 'binary_arg', but got " + value.getClass().getSimpleName()); case DOUBLE_ARG: if (value instanceof Double) { break; } throw new ClassCastException("Was expecting value of type Double for field 'double_arg', but got " + value.getClass().getSimpleName()); default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { _Fields setField = _Fields.findByThriftId(field.id); if (setField != null) { switch (setField) { case INT_ARG: if (field.type == INT_ARG_FIELD_DESC.type) { Integer int_arg; int_arg = iprot.readI32(); return int_arg; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case LONG_ARG: if (field.type == LONG_ARG_FIELD_DESC.type) { Long long_arg; long_arg = iprot.readI64(); return long_arg; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case STRING_ARG: if (field.type == STRING_ARG_FIELD_DESC.type) { String string_arg; string_arg = iprot.readString(); return string_arg; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case BOOL_ARG: if (field.type == BOOL_ARG_FIELD_DESC.type) { Boolean bool_arg; bool_arg = iprot.readBool(); return bool_arg; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case BINARY_ARG: if (field.type == BINARY_ARG_FIELD_DESC.type) { ByteBuffer binary_arg; binary_arg = iprot.readBinary(); return binary_arg; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } case DOUBLE_ARG: if (field.type == DOUBLE_ARG_FIELD_DESC.type) { Double double_arg; double_arg = iprot.readDouble(); return double_arg; } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } default: throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); return null; } } @Override protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { switch (setField_) { case INT_ARG: Integer int_arg = (Integer)value_; oprot.writeI32(int_arg); return; case LONG_ARG: Long long_arg = (Long)value_; oprot.writeI64(long_arg); return; case STRING_ARG: String string_arg = (String)value_; oprot.writeString(string_arg); return; case BOOL_ARG: Boolean bool_arg = (Boolean)value_; oprot.writeBool(bool_arg); return; case BINARY_ARG: ByteBuffer binary_arg = (ByteBuffer)value_; oprot.writeBinary(binary_arg); return; case DOUBLE_ARG: Double double_arg = (Double)value_; oprot.writeDouble(double_arg); return; default: throw new IllegalStateException("Cannot write union with unknown field " + setField_); } } @Override protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { switch (setField) { case INT_ARG: return INT_ARG_FIELD_DESC; case LONG_ARG: return LONG_ARG_FIELD_DESC; case STRING_ARG: return STRING_ARG_FIELD_DESC; case BOOL_ARG: return BOOL_ARG_FIELD_DESC; case BINARY_ARG: return BINARY_ARG_FIELD_DESC; case DOUBLE_ARG: return DOUBLE_ARG_FIELD_DESC; default: throw new IllegalArgumentException("Unknown field id " + setField); } } @Override protected org.apache.thrift7.protocol.TStruct getStructDesc() { return STRUCT_DESC; } @Override protected _Fields enumForId(short id) { return _Fields.findByThriftIdOrThrow(id); } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public int get_int_arg() { if (getSetField() == _Fields.INT_ARG) { return (Integer)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'int_arg' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_int_arg(int value) { setField_ = _Fields.INT_ARG; value_ = value; } public long get_long_arg() { if (getSetField() == _Fields.LONG_ARG) { return (Long)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'long_arg' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_long_arg(long value) { setField_ = _Fields.LONG_ARG; value_ = value; } public String get_string_arg() { if (getSetField() == _Fields.STRING_ARG) { return (String)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'string_arg' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_string_arg(String value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.STRING_ARG; value_ = value; } public boolean get_bool_arg() { if (getSetField() == _Fields.BOOL_ARG) { return (Boolean)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'bool_arg' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_bool_arg(boolean value) { setField_ = _Fields.BOOL_ARG; value_ = value; } public byte[] get_binary_arg() { set_binary_arg(org.apache.thrift7.TBaseHelper.rightSize(buffer_for_binary_arg())); ByteBuffer b = buffer_for_binary_arg(); return b == null ? null : b.array(); } public ByteBuffer buffer_for_binary_arg() { if (getSetField() == _Fields.BINARY_ARG) { return (ByteBuffer)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'binary_arg' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_binary_arg(byte[] value) { set_binary_arg(ByteBuffer.wrap(value)); } public void set_binary_arg(ByteBuffer value) { if (value == null) throw new NullPointerException(); setField_ = _Fields.BINARY_ARG; value_ = value; } public double get_double_arg() { if (getSetField() == _Fields.DOUBLE_ARG) { return (Double)getFieldValue(); } else { throw new RuntimeException("Cannot get field 'double_arg' because union is currently set to " + getFieldDesc(getSetField()).name); } } public void set_double_arg(double value) { setField_ = _Fields.DOUBLE_ARG; value_ = value; } public boolean is_set_int_arg() { return setField_ == _Fields.INT_ARG; } public boolean is_set_long_arg() { return setField_ == _Fields.LONG_ARG; } public boolean is_set_string_arg() { return setField_ == _Fields.STRING_ARG; } public boolean is_set_bool_arg() { return setField_ == _Fields.BOOL_ARG; } public boolean is_set_binary_arg() { return setField_ == _Fields.BINARY_ARG; } public boolean is_set_double_arg() { return setField_ == _Fields.DOUBLE_ARG; } public boolean equals(Object other) { if (other instanceof JavaObjectArg) { return equals((JavaObjectArg)other); } else { return false; } } public boolean equals(JavaObjectArg other) { return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); } @Override public int compareTo(JavaObjectArg other) { int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); if (lastComparison == 0) { return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); } return lastComparison; } @Override public int hashCode() { HashCodeBuilder hcb = new HashCodeBuilder(); hcb.append(this.getClass().getName()); org.apache.thrift7.TFieldIdEnum setField = getSetField(); if (setField != null) { hcb.append(setField.getThriftFieldId()); Object value = getFieldValue(); if (value instanceof org.apache.thrift7.TEnum) { hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); } else { hcb.append(value); } } return hcb.toHashCode(); } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/KillOptions.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class KillOptions implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("KillOptions"); private static final org.apache.thrift7.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("wait_secs", org.apache.thrift7.protocol.TType.I32, (short)1); private int wait_secs; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { WAIT_SECS((short)1, "wait_secs"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // WAIT_SECS return WAIT_SECS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __WAIT_SECS_ISSET_ID = 0; private BitSet __isset_bit_vector = new BitSet(1); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift7.meta_data.FieldMetaData("wait_secs", org.apache.thrift7.TFieldRequirementType.OPTIONAL, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(KillOptions.class, metaDataMap); } public KillOptions() { } /** * Performs a deep copy on other. */ public KillOptions(KillOptions other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); this.wait_secs = other.wait_secs; } public KillOptions deepCopy() { return new KillOptions(this); } @Override public void clear() { set_wait_secs_isSet(false); this.wait_secs = 0; } public int get_wait_secs() { return this.wait_secs; } public void set_wait_secs(int wait_secs) { this.wait_secs = wait_secs; set_wait_secs_isSet(true); } public void unset_wait_secs() { __isset_bit_vector.clear(__WAIT_SECS_ISSET_ID); } /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_wait_secs() { return __isset_bit_vector.get(__WAIT_SECS_ISSET_ID); } public void set_wait_secs_isSet(boolean value) { __isset_bit_vector.set(__WAIT_SECS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { switch (field) { case WAIT_SECS: if (value == null) { unset_wait_secs(); } else { set_wait_secs((Integer)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case WAIT_SECS: return Integer.valueOf(get_wait_secs()); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case WAIT_SECS: return is_set_wait_secs(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof KillOptions) return this.equals((KillOptions)that); return false; } public boolean equals(KillOptions that) { if (that == null) return false; boolean this_present_wait_secs = true && this.is_set_wait_secs(); boolean that_present_wait_secs = true && that.is_set_wait_secs(); if (this_present_wait_secs || that_present_wait_secs) { if (!(this_present_wait_secs && that_present_wait_secs)) return false; if (this.wait_secs != that.wait_secs) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_wait_secs = true && (is_set_wait_secs()); builder.append(present_wait_secs); if (present_wait_secs) builder.append(wait_secs); return builder.toHashCode(); } public int compareTo(KillOptions other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; KillOptions typedOther = (KillOptions)other; lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(typedOther.is_set_wait_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_wait_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.wait_secs, typedOther.wait_secs); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // WAIT_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.wait_secs = iprot.readI32(); set_wait_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (is_set_wait_secs()) { oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC); oprot.writeI32(this.wait_secs); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("KillOptions("); boolean first = true; if (is_set_wait_secs()) { sb.append("wait_secs:"); sb.append(this.wait_secs); first = false; } sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/Nimbus.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class Nimbus { public interface Iface { public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException; public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException; public void killTopology(String name) throws NotAliveException, org.apache.thrift7.TException; public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift7.TException; public void activate(String name) throws NotAliveException, org.apache.thrift7.TException; public void deactivate(String name) throws NotAliveException, org.apache.thrift7.TException; public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException; public String beginFileUpload() throws org.apache.thrift7.TException; public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift7.TException; public void finishFileUpload(String location) throws org.apache.thrift7.TException; public String beginFileDownload(String file) throws org.apache.thrift7.TException; public ByteBuffer downloadChunk(String id) throws org.apache.thrift7.TException; public String getNimbusConf() throws org.apache.thrift7.TException; public ClusterSummary getClusterInfo() throws org.apache.thrift7.TException; public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift7.TException; public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift7.TException; public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift7.TException; public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift7.TException; } public interface AsyncIface { public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void killTopology(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void activate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void deactivate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void rebalance(String name, RebalanceOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void beginFileUpload(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void finishFileUpload(String location, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void beginFileDownload(String file, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void downloadChunk(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void getNimbusConf(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void getClusterInfo(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void getTopologyInfo(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void getTopologyConf(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void getTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; public void getUserTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; } public static class Client extends org.apache.thrift7.TServiceClient implements Iface { public static class Factory implements org.apache.thrift7.TServiceClientFactory { public Factory() {} public Client getClient(org.apache.thrift7.protocol.TProtocol prot) { return new Client(prot); } public Client getClient(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { return new Client(iprot, oprot); } } public Client(org.apache.thrift7.protocol.TProtocol prot) { super(prot, prot); } public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { super(iprot, oprot); } public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException { send_submitTopology(name, uploadedJarLocation, jsonConf, topology); recv_submitTopology(); } public void send_submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws org.apache.thrift7.TException { submitTopology_args args = new submitTopology_args(); args.set_name(name); args.set_uploadedJarLocation(uploadedJarLocation); args.set_jsonConf(jsonConf); args.set_topology(topology); sendBase("submitTopology", args); } public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException { submitTopology_result result = new submitTopology_result(); receiveBase(result, "submitTopology"); if (result.e != null) { throw result.e; } if (result.ite != null) { throw result.ite; } return; } public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException { send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options); recv_submitTopologyWithOpts(); } public void send_submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws org.apache.thrift7.TException { submitTopologyWithOpts_args args = new submitTopologyWithOpts_args(); args.set_name(name); args.set_uploadedJarLocation(uploadedJarLocation); args.set_jsonConf(jsonConf); args.set_topology(topology); args.set_options(options); sendBase("submitTopologyWithOpts", args); } public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException { submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); receiveBase(result, "submitTopologyWithOpts"); if (result.e != null) { throw result.e; } if (result.ite != null) { throw result.ite; } return; } public void killTopology(String name) throws NotAliveException, org.apache.thrift7.TException { send_killTopology(name); recv_killTopology(); } public void send_killTopology(String name) throws org.apache.thrift7.TException { killTopology_args args = new killTopology_args(); args.set_name(name); sendBase("killTopology", args); } public void recv_killTopology() throws NotAliveException, org.apache.thrift7.TException { killTopology_result result = new killTopology_result(); receiveBase(result, "killTopology"); if (result.e != null) { throw result.e; } return; } public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift7.TException { send_killTopologyWithOpts(name, options); recv_killTopologyWithOpts(); } public void send_killTopologyWithOpts(String name, KillOptions options) throws org.apache.thrift7.TException { killTopologyWithOpts_args args = new killTopologyWithOpts_args(); args.set_name(name); args.set_options(options); sendBase("killTopologyWithOpts", args); } public void recv_killTopologyWithOpts() throws NotAliveException, org.apache.thrift7.TException { killTopologyWithOpts_result result = new killTopologyWithOpts_result(); receiveBase(result, "killTopologyWithOpts"); if (result.e != null) { throw result.e; } return; } public void activate(String name) throws NotAliveException, org.apache.thrift7.TException { send_activate(name); recv_activate(); } public void send_activate(String name) throws org.apache.thrift7.TException { activate_args args = new activate_args(); args.set_name(name); sendBase("activate", args); } public void recv_activate() throws NotAliveException, org.apache.thrift7.TException { activate_result result = new activate_result(); receiveBase(result, "activate"); if (result.e != null) { throw result.e; } return; } public void deactivate(String name) throws NotAliveException, org.apache.thrift7.TException { send_deactivate(name); recv_deactivate(); } public void send_deactivate(String name) throws org.apache.thrift7.TException { deactivate_args args = new deactivate_args(); args.set_name(name); sendBase("deactivate", args); } public void recv_deactivate() throws NotAliveException, org.apache.thrift7.TException { deactivate_result result = new deactivate_result(); receiveBase(result, "deactivate"); if (result.e != null) { throw result.e; } return; } public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException { send_rebalance(name, options); recv_rebalance(); } public void send_rebalance(String name, RebalanceOptions options) throws org.apache.thrift7.TException { rebalance_args args = new rebalance_args(); args.set_name(name); args.set_options(options); sendBase("rebalance", args); } public void recv_rebalance() throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException { rebalance_result result = new rebalance_result(); receiveBase(result, "rebalance"); if (result.e != null) { throw result.e; } if (result.ite != null) { throw result.ite; } return; } public String beginFileUpload() throws org.apache.thrift7.TException { send_beginFileUpload(); return recv_beginFileUpload(); } public void send_beginFileUpload() throws org.apache.thrift7.TException { beginFileUpload_args args = new beginFileUpload_args(); sendBase("beginFileUpload", args); } public String recv_beginFileUpload() throws org.apache.thrift7.TException { beginFileUpload_result result = new beginFileUpload_result(); receiveBase(result, "beginFileUpload"); if (result.is_set_success()) { return result.success; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); } public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift7.TException { send_uploadChunk(location, chunk); recv_uploadChunk(); } public void send_uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift7.TException { uploadChunk_args args = new uploadChunk_args(); args.set_location(location); args.set_chunk(chunk); sendBase("uploadChunk", args); } public void recv_uploadChunk() throws org.apache.thrift7.TException { uploadChunk_result result = new uploadChunk_result(); receiveBase(result, "uploadChunk"); return; } public void finishFileUpload(String location) throws org.apache.thrift7.TException { send_finishFileUpload(location); recv_finishFileUpload(); } public void send_finishFileUpload(String location) throws org.apache.thrift7.TException { finishFileUpload_args args = new finishFileUpload_args(); args.set_location(location); sendBase("finishFileUpload", args); } public void recv_finishFileUpload() throws org.apache.thrift7.TException { finishFileUpload_result result = new finishFileUpload_result(); receiveBase(result, "finishFileUpload"); return; } public String beginFileDownload(String file) throws org.apache.thrift7.TException { send_beginFileDownload(file); return recv_beginFileDownload(); } public void send_beginFileDownload(String file) throws org.apache.thrift7.TException { beginFileDownload_args args = new beginFileDownload_args(); args.set_file(file); sendBase("beginFileDownload", args); } public String recv_beginFileDownload() throws org.apache.thrift7.TException { beginFileDownload_result result = new beginFileDownload_result(); receiveBase(result, "beginFileDownload"); if (result.is_set_success()) { return result.success; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); } public ByteBuffer downloadChunk(String id) throws org.apache.thrift7.TException { send_downloadChunk(id); return recv_downloadChunk(); } public void send_downloadChunk(String id) throws org.apache.thrift7.TException { downloadChunk_args args = new downloadChunk_args(); args.set_id(id); sendBase("downloadChunk", args); } public ByteBuffer recv_downloadChunk() throws org.apache.thrift7.TException { downloadChunk_result result = new downloadChunk_result(); receiveBase(result, "downloadChunk"); if (result.is_set_success()) { return result.success; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); } public String getNimbusConf() throws org.apache.thrift7.TException { send_getNimbusConf(); return recv_getNimbusConf(); } public void send_getNimbusConf() throws org.apache.thrift7.TException { getNimbusConf_args args = new getNimbusConf_args(); sendBase("getNimbusConf", args); } public String recv_getNimbusConf() throws org.apache.thrift7.TException { getNimbusConf_result result = new getNimbusConf_result(); receiveBase(result, "getNimbusConf"); if (result.is_set_success()) { return result.success; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); } public ClusterSummary getClusterInfo() throws org.apache.thrift7.TException { send_getClusterInfo(); return recv_getClusterInfo(); } public void send_getClusterInfo() throws org.apache.thrift7.TException { getClusterInfo_args args = new getClusterInfo_args(); sendBase("getClusterInfo", args); } public ClusterSummary recv_getClusterInfo() throws org.apache.thrift7.TException { getClusterInfo_result result = new getClusterInfo_result(); receiveBase(result, "getClusterInfo"); if (result.is_set_success()) { return result.success; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); } public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift7.TException { send_getTopologyInfo(id); return recv_getTopologyInfo(); } public void send_getTopologyInfo(String id) throws org.apache.thrift7.TException { getTopologyInfo_args args = new getTopologyInfo_args(); args.set_id(id); sendBase("getTopologyInfo", args); } public TopologyInfo recv_getTopologyInfo() throws NotAliveException, org.apache.thrift7.TException { getTopologyInfo_result result = new getTopologyInfo_result(); receiveBase(result, "getTopologyInfo"); if (result.is_set_success()) { return result.success; } if (result.e != null) { throw result.e; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); } public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift7.TException { send_getTopologyConf(id); return recv_getTopologyConf(); } public void send_getTopologyConf(String id) throws org.apache.thrift7.TException { getTopologyConf_args args = new getTopologyConf_args(); args.set_id(id); sendBase("getTopologyConf", args); } public String recv_getTopologyConf() throws NotAliveException, org.apache.thrift7.TException { getTopologyConf_result result = new getTopologyConf_result(); receiveBase(result, "getTopologyConf"); if (result.is_set_success()) { return result.success; } if (result.e != null) { throw result.e; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); } public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift7.TException { send_getTopology(id); return recv_getTopology(); } public void send_getTopology(String id) throws org.apache.thrift7.TException { getTopology_args args = new getTopology_args(); args.set_id(id); sendBase("getTopology", args); } public StormTopology recv_getTopology() throws NotAliveException, org.apache.thrift7.TException { getTopology_result result = new getTopology_result(); receiveBase(result, "getTopology"); if (result.is_set_success()) { return result.success; } if (result.e != null) { throw result.e; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); } public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift7.TException { send_getUserTopology(id); return recv_getUserTopology(); } public void send_getUserTopology(String id) throws org.apache.thrift7.TException { getUserTopology_args args = new getUserTopology_args(); args.set_id(id); sendBase("getUserTopology", args); } public StormTopology recv_getUserTopology() throws NotAliveException, org.apache.thrift7.TException { getUserTopology_result result = new getUserTopology_result(); receiveBase(result, "getUserTopology"); if (result.is_set_success()) { return result.success; } if (result.e != null) { throw result.e; } throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); } } public static class AsyncClient extends org.apache.thrift7.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift7.async.TAsyncClientFactory { private org.apache.thrift7.async.TAsyncClientManager clientManager; private org.apache.thrift7.protocol.TProtocolFactory protocolFactory; public Factory(org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.protocol.TProtocolFactory protocolFactory) { this.clientManager = clientManager; this.protocolFactory = protocolFactory; } public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNonblockingTransport transport) { return new AsyncClient(protocolFactory, clientManager, transport); } } public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.transport.TNonblockingTransport transport) { super(protocolFactory, clientManager, transport); } public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); submitTopology_call method_call = new submitTopology_call(name, uploadedJarLocation, jsonConf, topology, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class submitTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { private String name; private String uploadedJarLocation; private String jsonConf; private StormTopology topology; public submitTopology_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.name = name; this.uploadedJarLocation = uploadedJarLocation; this.jsonConf = jsonConf; this.topology = topology; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("submitTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); submitTopology_args args = new submitTopology_args(); args.set_name(name); args.set_uploadedJarLocation(uploadedJarLocation); args.set_jsonConf(jsonConf); args.set_topology(topology); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_submitTopology(); } } public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); submitTopologyWithOpts_call method_call = new submitTopologyWithOpts_call(name, uploadedJarLocation, jsonConf, topology, options, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class submitTopologyWithOpts_call extends org.apache.thrift7.async.TAsyncMethodCall { private String name; private String uploadedJarLocation; private String jsonConf; private StormTopology topology; private SubmitOptions options; public submitTopologyWithOpts_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.name = name; this.uploadedJarLocation = uploadedJarLocation; this.jsonConf = jsonConf; this.topology = topology; this.options = options; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("submitTopologyWithOpts", org.apache.thrift7.protocol.TMessageType.CALL, 0)); submitTopologyWithOpts_args args = new submitTopologyWithOpts_args(); args.set_name(name); args.set_uploadedJarLocation(uploadedJarLocation); args.set_jsonConf(jsonConf); args.set_topology(topology); args.set_options(options); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_submitTopologyWithOpts(); } } public void killTopology(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); killTopology_call method_call = new killTopology_call(name, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class killTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { private String name; public killTopology_call(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.name = name; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("killTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); killTopology_args args = new killTopology_args(); args.set_name(name); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_killTopology(); } } public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); killTopologyWithOpts_call method_call = new killTopologyWithOpts_call(name, options, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class killTopologyWithOpts_call extends org.apache.thrift7.async.TAsyncMethodCall { private String name; private KillOptions options; public killTopologyWithOpts_call(String name, KillOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.name = name; this.options = options; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("killTopologyWithOpts", org.apache.thrift7.protocol.TMessageType.CALL, 0)); killTopologyWithOpts_args args = new killTopologyWithOpts_args(); args.set_name(name); args.set_options(options); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_killTopologyWithOpts(); } } public void activate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); activate_call method_call = new activate_call(name, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class activate_call extends org.apache.thrift7.async.TAsyncMethodCall { private String name; public activate_call(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.name = name; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("activate", org.apache.thrift7.protocol.TMessageType.CALL, 0)); activate_args args = new activate_args(); args.set_name(name); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_activate(); } } public void deactivate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); deactivate_call method_call = new deactivate_call(name, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class deactivate_call extends org.apache.thrift7.async.TAsyncMethodCall { private String name; public deactivate_call(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.name = name; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("deactivate", org.apache.thrift7.protocol.TMessageType.CALL, 0)); deactivate_args args = new deactivate_args(); args.set_name(name); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_deactivate(); } } public void rebalance(String name, RebalanceOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); rebalance_call method_call = new rebalance_call(name, options, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class rebalance_call extends org.apache.thrift7.async.TAsyncMethodCall { private String name; private RebalanceOptions options; public rebalance_call(String name, RebalanceOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.name = name; this.options = options; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("rebalance", org.apache.thrift7.protocol.TMessageType.CALL, 0)); rebalance_args args = new rebalance_args(); args.set_name(name); args.set_options(options); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_rebalance(); } } public void beginFileUpload(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class beginFileUpload_call extends org.apache.thrift7.async.TAsyncMethodCall { public beginFileUpload_call(org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("beginFileUpload", org.apache.thrift7.protocol.TMessageType.CALL, 0)); beginFileUpload_args args = new beginFileUpload_args(); args.write(prot); prot.writeMessageEnd(); } public String getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_beginFileUpload(); } } public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); uploadChunk_call method_call = new uploadChunk_call(location, chunk, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class uploadChunk_call extends org.apache.thrift7.async.TAsyncMethodCall { private String location; private ByteBuffer chunk; public uploadChunk_call(String location, ByteBuffer chunk, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.location = location; this.chunk = chunk; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("uploadChunk", org.apache.thrift7.protocol.TMessageType.CALL, 0)); uploadChunk_args args = new uploadChunk_args(); args.set_location(location); args.set_chunk(chunk); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_uploadChunk(); } } public void finishFileUpload(String location, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); finishFileUpload_call method_call = new finishFileUpload_call(location, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class finishFileUpload_call extends org.apache.thrift7.async.TAsyncMethodCall { private String location; public finishFileUpload_call(String location, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.location = location; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("finishFileUpload", org.apache.thrift7.protocol.TMessageType.CALL, 0)); finishFileUpload_args args = new finishFileUpload_args(); args.set_location(location); args.write(prot); prot.writeMessageEnd(); } public void getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); (new Client(prot)).recv_finishFileUpload(); } } public void beginFileDownload(String file, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); beginFileDownload_call method_call = new beginFileDownload_call(file, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class beginFileDownload_call extends org.apache.thrift7.async.TAsyncMethodCall { private String file; public beginFileDownload_call(String file, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.file = file; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("beginFileDownload", org.apache.thrift7.protocol.TMessageType.CALL, 0)); beginFileDownload_args args = new beginFileDownload_args(); args.set_file(file); args.write(prot); prot.writeMessageEnd(); } public String getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_beginFileDownload(); } } public void downloadChunk(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); downloadChunk_call method_call = new downloadChunk_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class downloadChunk_call extends org.apache.thrift7.async.TAsyncMethodCall { private String id; public downloadChunk_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.id = id; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("downloadChunk", org.apache.thrift7.protocol.TMessageType.CALL, 0)); downloadChunk_args args = new downloadChunk_args(); args.set_id(id); args.write(prot); prot.writeMessageEnd(); } public ByteBuffer getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_downloadChunk(); } } public void getNimbusConf(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); getNimbusConf_call method_call = new getNimbusConf_call(resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class getNimbusConf_call extends org.apache.thrift7.async.TAsyncMethodCall { public getNimbusConf_call(org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getNimbusConf", org.apache.thrift7.protocol.TMessageType.CALL, 0)); getNimbusConf_args args = new getNimbusConf_args(); args.write(prot); prot.writeMessageEnd(); } public String getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_getNimbusConf(); } } public void getClusterInfo(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); getClusterInfo_call method_call = new getClusterInfo_call(resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class getClusterInfo_call extends org.apache.thrift7.async.TAsyncMethodCall { public getClusterInfo_call(org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getClusterInfo", org.apache.thrift7.protocol.TMessageType.CALL, 0)); getClusterInfo_args args = new getClusterInfo_args(); args.write(prot); prot.writeMessageEnd(); } public ClusterSummary getResult() throws org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_getClusterInfo(); } } public void getTopologyInfo(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); getTopologyInfo_call method_call = new getTopologyInfo_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class getTopologyInfo_call extends org.apache.thrift7.async.TAsyncMethodCall { private String id; public getTopologyInfo_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.id = id; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopologyInfo", org.apache.thrift7.protocol.TMessageType.CALL, 0)); getTopologyInfo_args args = new getTopologyInfo_args(); args.set_id(id); args.write(prot); prot.writeMessageEnd(); } public TopologyInfo getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_getTopologyInfo(); } } public void getTopologyConf(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); getTopologyConf_call method_call = new getTopologyConf_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class getTopologyConf_call extends org.apache.thrift7.async.TAsyncMethodCall { private String id; public getTopologyConf_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.id = id; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopologyConf", org.apache.thrift7.protocol.TMessageType.CALL, 0)); getTopologyConf_args args = new getTopologyConf_args(); args.set_id(id); args.write(prot); prot.writeMessageEnd(); } public String getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_getTopologyConf(); } } public void getTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); getTopology_call method_call = new getTopology_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class getTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { private String id; public getTopology_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.id = id; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); getTopology_args args = new getTopology_args(); args.set_id(id); args.write(prot); prot.writeMessageEnd(); } public StormTopology getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_getTopology(); } } public void getUserTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { checkReady(); getUserTopology_call method_call = new getUserTopology_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class getUserTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { private String id; public getUserTopology_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { super(client, protocolFactory, transport, resultHandler, false); this.id = id; } public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getUserTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); getUserTopology_args args = new getUserTopology_args(); args.set_id(id); args.write(prot); prot.writeMessageEnd(); } public StormTopology getResult() throws NotAliveException, org.apache.thrift7.TException { if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); return (new Client(prot)).recv_getUserTopology(); } } } public static class Processor extends org.apache.thrift7.TBaseProcessor implements org.apache.thrift7.TProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); public Processor(I iface) { super(iface, getProcessMap(new HashMap>())); } protected Processor(I iface, Map> processMap) { super(iface, getProcessMap(processMap)); } private static Map> getProcessMap(Map> processMap) { processMap.put("submitTopology", new submitTopology()); processMap.put("submitTopologyWithOpts", new submitTopologyWithOpts()); processMap.put("killTopology", new killTopology()); processMap.put("killTopologyWithOpts", new killTopologyWithOpts()); processMap.put("activate", new activate()); processMap.put("deactivate", new deactivate()); processMap.put("rebalance", new rebalance()); processMap.put("beginFileUpload", new beginFileUpload()); processMap.put("uploadChunk", new uploadChunk()); processMap.put("finishFileUpload", new finishFileUpload()); processMap.put("beginFileDownload", new beginFileDownload()); processMap.put("downloadChunk", new downloadChunk()); processMap.put("getNimbusConf", new getNimbusConf()); processMap.put("getClusterInfo", new getClusterInfo()); processMap.put("getTopologyInfo", new getTopologyInfo()); processMap.put("getTopologyConf", new getTopologyConf()); processMap.put("getTopology", new getTopology()); processMap.put("getUserTopology", new getUserTopology()); return processMap; } private static class submitTopology extends org.apache.thrift7.ProcessFunction { public submitTopology() { super("submitTopology"); } protected submitTopology_args getEmptyArgsInstance() { return new submitTopology_args(); } protected submitTopology_result getResult(I iface, submitTopology_args args) throws org.apache.thrift7.TException { submitTopology_result result = new submitTopology_result(); try { iface.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology); } catch (AlreadyAliveException e) { result.e = e; } catch (InvalidTopologyException ite) { result.ite = ite; } return result; } } private static class submitTopologyWithOpts extends org.apache.thrift7.ProcessFunction { public submitTopologyWithOpts() { super("submitTopologyWithOpts"); } protected submitTopologyWithOpts_args getEmptyArgsInstance() { return new submitTopologyWithOpts_args(); } protected submitTopologyWithOpts_result getResult(I iface, submitTopologyWithOpts_args args) throws org.apache.thrift7.TException { submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); try { iface.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options); } catch (AlreadyAliveException e) { result.e = e; } catch (InvalidTopologyException ite) { result.ite = ite; } return result; } } private static class killTopology extends org.apache.thrift7.ProcessFunction { public killTopology() { super("killTopology"); } protected killTopology_args getEmptyArgsInstance() { return new killTopology_args(); } protected killTopology_result getResult(I iface, killTopology_args args) throws org.apache.thrift7.TException { killTopology_result result = new killTopology_result(); try { iface.killTopology(args.name); } catch (NotAliveException e) { result.e = e; } return result; } } private static class killTopologyWithOpts extends org.apache.thrift7.ProcessFunction { public killTopologyWithOpts() { super("killTopologyWithOpts"); } protected killTopologyWithOpts_args getEmptyArgsInstance() { return new killTopologyWithOpts_args(); } protected killTopologyWithOpts_result getResult(I iface, killTopologyWithOpts_args args) throws org.apache.thrift7.TException { killTopologyWithOpts_result result = new killTopologyWithOpts_result(); try { iface.killTopologyWithOpts(args.name, args.options); } catch (NotAliveException e) { result.e = e; } return result; } } private static class activate extends org.apache.thrift7.ProcessFunction { public activate() { super("activate"); } protected activate_args getEmptyArgsInstance() { return new activate_args(); } protected activate_result getResult(I iface, activate_args args) throws org.apache.thrift7.TException { activate_result result = new activate_result(); try { iface.activate(args.name); } catch (NotAliveException e) { result.e = e; } return result; } } private static class deactivate extends org.apache.thrift7.ProcessFunction { public deactivate() { super("deactivate"); } protected deactivate_args getEmptyArgsInstance() { return new deactivate_args(); } protected deactivate_result getResult(I iface, deactivate_args args) throws org.apache.thrift7.TException { deactivate_result result = new deactivate_result(); try { iface.deactivate(args.name); } catch (NotAliveException e) { result.e = e; } return result; } } private static class rebalance extends org.apache.thrift7.ProcessFunction { public rebalance() { super("rebalance"); } protected rebalance_args getEmptyArgsInstance() { return new rebalance_args(); } protected rebalance_result getResult(I iface, rebalance_args args) throws org.apache.thrift7.TException { rebalance_result result = new rebalance_result(); try { iface.rebalance(args.name, args.options); } catch (NotAliveException e) { result.e = e; } catch (InvalidTopologyException ite) { result.ite = ite; } return result; } } private static class beginFileUpload extends org.apache.thrift7.ProcessFunction { public beginFileUpload() { super("beginFileUpload"); } protected beginFileUpload_args getEmptyArgsInstance() { return new beginFileUpload_args(); } protected beginFileUpload_result getResult(I iface, beginFileUpload_args args) throws org.apache.thrift7.TException { beginFileUpload_result result = new beginFileUpload_result(); result.success = iface.beginFileUpload(); return result; } } private static class uploadChunk extends org.apache.thrift7.ProcessFunction { public uploadChunk() { super("uploadChunk"); } protected uploadChunk_args getEmptyArgsInstance() { return new uploadChunk_args(); } protected uploadChunk_result getResult(I iface, uploadChunk_args args) throws org.apache.thrift7.TException { uploadChunk_result result = new uploadChunk_result(); iface.uploadChunk(args.location, args.chunk); return result; } } private static class finishFileUpload extends org.apache.thrift7.ProcessFunction { public finishFileUpload() { super("finishFileUpload"); } protected finishFileUpload_args getEmptyArgsInstance() { return new finishFileUpload_args(); } protected finishFileUpload_result getResult(I iface, finishFileUpload_args args) throws org.apache.thrift7.TException { finishFileUpload_result result = new finishFileUpload_result(); iface.finishFileUpload(args.location); return result; } } private static class beginFileDownload extends org.apache.thrift7.ProcessFunction { public beginFileDownload() { super("beginFileDownload"); } protected beginFileDownload_args getEmptyArgsInstance() { return new beginFileDownload_args(); } protected beginFileDownload_result getResult(I iface, beginFileDownload_args args) throws org.apache.thrift7.TException { beginFileDownload_result result = new beginFileDownload_result(); result.success = iface.beginFileDownload(args.file); return result; } } private static class downloadChunk extends org.apache.thrift7.ProcessFunction { public downloadChunk() { super("downloadChunk"); } protected downloadChunk_args getEmptyArgsInstance() { return new downloadChunk_args(); } protected downloadChunk_result getResult(I iface, downloadChunk_args args) throws org.apache.thrift7.TException { downloadChunk_result result = new downloadChunk_result(); result.success = iface.downloadChunk(args.id); return result; } } private static class getNimbusConf extends org.apache.thrift7.ProcessFunction { public getNimbusConf() { super("getNimbusConf"); } protected getNimbusConf_args getEmptyArgsInstance() { return new getNimbusConf_args(); } protected getNimbusConf_result getResult(I iface, getNimbusConf_args args) throws org.apache.thrift7.TException { getNimbusConf_result result = new getNimbusConf_result(); result.success = iface.getNimbusConf(); return result; } } private static class getClusterInfo extends org.apache.thrift7.ProcessFunction { public getClusterInfo() { super("getClusterInfo"); } protected getClusterInfo_args getEmptyArgsInstance() { return new getClusterInfo_args(); } protected getClusterInfo_result getResult(I iface, getClusterInfo_args args) throws org.apache.thrift7.TException { getClusterInfo_result result = new getClusterInfo_result(); result.success = iface.getClusterInfo(); return result; } } private static class getTopologyInfo extends org.apache.thrift7.ProcessFunction { public getTopologyInfo() { super("getTopologyInfo"); } protected getTopologyInfo_args getEmptyArgsInstance() { return new getTopologyInfo_args(); } protected getTopologyInfo_result getResult(I iface, getTopologyInfo_args args) throws org.apache.thrift7.TException { getTopologyInfo_result result = new getTopologyInfo_result(); try { result.success = iface.getTopologyInfo(args.id); } catch (NotAliveException e) { result.e = e; } return result; } } private static class getTopologyConf extends org.apache.thrift7.ProcessFunction { public getTopologyConf() { super("getTopologyConf"); } protected getTopologyConf_args getEmptyArgsInstance() { return new getTopologyConf_args(); } protected getTopologyConf_result getResult(I iface, getTopologyConf_args args) throws org.apache.thrift7.TException { getTopologyConf_result result = new getTopologyConf_result(); try { result.success = iface.getTopologyConf(args.id); } catch (NotAliveException e) { result.e = e; } return result; } } private static class getTopology extends org.apache.thrift7.ProcessFunction { public getTopology() { super("getTopology"); } protected getTopology_args getEmptyArgsInstance() { return new getTopology_args(); } protected getTopology_result getResult(I iface, getTopology_args args) throws org.apache.thrift7.TException { getTopology_result result = new getTopology_result(); try { result.success = iface.getTopology(args.id); } catch (NotAliveException e) { result.e = e; } return result; } } private static class getUserTopology extends org.apache.thrift7.ProcessFunction { public getUserTopology() { super("getUserTopology"); } protected getUserTopology_args getEmptyArgsInstance() { return new getUserTopology_args(); } protected getUserTopology_result getResult(I iface, getUserTopology_args args) throws org.apache.thrift7.TException { getUserTopology_result result = new getUserTopology_result(); try { result.success = iface.getUserTopology(args.id); } catch (NotAliveException e) { result.e = e; } return result; } } } public static class submitTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopology_args"); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField UPLOADED_JAR_LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("uploadedJarLocation", org.apache.thrift7.protocol.TType.STRING, (short)2); private static final org.apache.thrift7.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift7.protocol.TField("jsonConf", org.apache.thrift7.protocol.TType.STRING, (short)3); private static final org.apache.thrift7.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift7.protocol.TField("topology", org.apache.thrift7.protocol.TType.STRUCT, (short)4); private String name; // required private String uploadedJarLocation; // required private String jsonConf; // required private StormTopology topology; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { NAME((short)1, "name"), UPLOADED_JAR_LOCATION((short)2, "uploadedJarLocation"), JSON_CONF((short)3, "jsonConf"), TOPOLOGY((short)4, "topology"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // NAME return NAME; case 2: // UPLOADED_JAR_LOCATION return UPLOADED_JAR_LOCATION; case 3: // JSON_CONF return JSON_CONF; case 4: // TOPOLOGY return TOPOLOGY; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.UPLOADED_JAR_LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("uploadedJarLocation", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift7.meta_data.FieldMetaData("jsonConf", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift7.meta_data.FieldMetaData("topology", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopology_args.class, metaDataMap); } public submitTopology_args() { } public submitTopology_args( String name, String uploadedJarLocation, String jsonConf, StormTopology topology) { this(); this.name = name; this.uploadedJarLocation = uploadedJarLocation; this.jsonConf = jsonConf; this.topology = topology; } /** * Performs a deep copy on other. */ public submitTopology_args(submitTopology_args other) { if (other.is_set_name()) { this.name = other.name; } if (other.is_set_uploadedJarLocation()) { this.uploadedJarLocation = other.uploadedJarLocation; } if (other.is_set_jsonConf()) { this.jsonConf = other.jsonConf; } if (other.is_set_topology()) { this.topology = new StormTopology(other.topology); } } public submitTopology_args deepCopy() { return new submitTopology_args(this); } @Override public void clear() { this.name = null; this.uploadedJarLocation = null; this.jsonConf = null; this.topology = null; } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public String get_uploadedJarLocation() { return this.uploadedJarLocation; } public void set_uploadedJarLocation(String uploadedJarLocation) { this.uploadedJarLocation = uploadedJarLocation; } public void unset_uploadedJarLocation() { this.uploadedJarLocation = null; } /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ public boolean is_set_uploadedJarLocation() { return this.uploadedJarLocation != null; } public void set_uploadedJarLocation_isSet(boolean value) { if (!value) { this.uploadedJarLocation = null; } } public String get_jsonConf() { return this.jsonConf; } public void set_jsonConf(String jsonConf) { this.jsonConf = jsonConf; } public void unset_jsonConf() { this.jsonConf = null; } /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ public boolean is_set_jsonConf() { return this.jsonConf != null; } public void set_jsonConf_isSet(boolean value) { if (!value) { this.jsonConf = null; } } public StormTopology get_topology() { return this.topology; } public void set_topology(StormTopology topology) { this.topology = topology; } public void unset_topology() { this.topology = null; } /** Returns true if field topology is set (has been assigned a value) and false otherwise */ public boolean is_set_topology() { return this.topology != null; } public void set_topology_isSet(boolean value) { if (!value) { this.topology = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; case UPLOADED_JAR_LOCATION: if (value == null) { unset_uploadedJarLocation(); } else { set_uploadedJarLocation((String)value); } break; case JSON_CONF: if (value == null) { unset_jsonConf(); } else { set_jsonConf((String)value); } break; case TOPOLOGY: if (value == null) { unset_topology(); } else { set_topology((StormTopology)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case NAME: return get_name(); case UPLOADED_JAR_LOCATION: return get_uploadedJarLocation(); case JSON_CONF: return get_jsonConf(); case TOPOLOGY: return get_topology(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case NAME: return is_set_name(); case UPLOADED_JAR_LOCATION: return is_set_uploadedJarLocation(); case JSON_CONF: return is_set_jsonConf(); case TOPOLOGY: return is_set_topology(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof submitTopology_args) return this.equals((submitTopology_args)that); return false; } public boolean equals(submitTopology_args that) { if (that == null) return false; boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } boolean this_present_uploadedJarLocation = true && this.is_set_uploadedJarLocation(); boolean that_present_uploadedJarLocation = true && that.is_set_uploadedJarLocation(); if (this_present_uploadedJarLocation || that_present_uploadedJarLocation) { if (!(this_present_uploadedJarLocation && that_present_uploadedJarLocation)) return false; if (!this.uploadedJarLocation.equals(that.uploadedJarLocation)) return false; } boolean this_present_jsonConf = true && this.is_set_jsonConf(); boolean that_present_jsonConf = true && that.is_set_jsonConf(); if (this_present_jsonConf || that_present_jsonConf) { if (!(this_present_jsonConf && that_present_jsonConf)) return false; if (!this.jsonConf.equals(that.jsonConf)) return false; } boolean this_present_topology = true && this.is_set_topology(); boolean that_present_topology = true && that.is_set_topology(); if (this_present_topology || that_present_topology) { if (!(this_present_topology && that_present_topology)) return false; if (!this.topology.equals(that.topology)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); boolean present_uploadedJarLocation = true && (is_set_uploadedJarLocation()); builder.append(present_uploadedJarLocation); if (present_uploadedJarLocation) builder.append(uploadedJarLocation); boolean present_jsonConf = true && (is_set_jsonConf()); builder.append(present_jsonConf); if (present_jsonConf) builder.append(jsonConf); boolean present_topology = true && (is_set_topology()); builder.append(present_topology); if (present_topology) builder.append(topology); return builder.toHashCode(); } public int compareTo(submitTopology_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; submitTopology_args typedOther = (submitTopology_args)other; lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_uploadedJarLocation()).compareTo(typedOther.is_set_uploadedJarLocation()); if (lastComparison != 0) { return lastComparison; } if (is_set_uploadedJarLocation()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uploadedJarLocation, typedOther.uploadedJarLocation); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(typedOther.is_set_jsonConf()); if (lastComparison != 0) { return lastComparison; } if (is_set_jsonConf()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_topology()).compareTo(typedOther.is_set_topology()); if (lastComparison != 0) { return lastComparison; } if (is_set_topology()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topology, typedOther.topology); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // UPLOADED_JAR_LOCATION if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.uploadedJarLocation = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // JSON_CONF if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.jsonConf = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // TOPOLOGY if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.topology = new StormTopology(); this.topology.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } if (this.uploadedJarLocation != null) { oprot.writeFieldBegin(UPLOADED_JAR_LOCATION_FIELD_DESC); oprot.writeString(this.uploadedJarLocation); oprot.writeFieldEnd(); } if (this.jsonConf != null) { oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); oprot.writeString(this.jsonConf); oprot.writeFieldEnd(); } if (this.topology != null) { oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); this.topology.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("submitTopology_args("); boolean first = true; sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; if (!first) sb.append(", "); sb.append("uploadedJarLocation:"); if (this.uploadedJarLocation == null) { sb.append("null"); } else { sb.append(this.uploadedJarLocation); } first = false; if (!first) sb.append(", "); sb.append("jsonConf:"); if (this.jsonConf == null) { sb.append("null"); } else { sb.append(this.jsonConf); } first = false; if (!first) sb.append(", "); sb.append("topology:"); if (this.topology == null) { sb.append("null"); } else { sb.append(this.topology); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class submitTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopology_result"); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField ITE_FIELD_DESC = new org.apache.thrift7.protocol.TField("ite", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private AlreadyAliveException e; // required private InvalidTopologyException ite; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { E((short)1, "e"), ITE((short)2, "ite"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; case 2: // ITE return ITE; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); tmpMap.put(_Fields.ITE, new org.apache.thrift7.meta_data.FieldMetaData("ite", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopology_result.class, metaDataMap); } public submitTopology_result() { } public submitTopology_result( AlreadyAliveException e, InvalidTopologyException ite) { this(); this.e = e; this.ite = ite; } /** * Performs a deep copy on other. */ public submitTopology_result(submitTopology_result other) { if (other.is_set_e()) { this.e = new AlreadyAliveException(other.e); } if (other.is_set_ite()) { this.ite = new InvalidTopologyException(other.ite); } } public submitTopology_result deepCopy() { return new submitTopology_result(this); } @Override public void clear() { this.e = null; this.ite = null; } public AlreadyAliveException get_e() { return this.e; } public void set_e(AlreadyAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public InvalidTopologyException get_ite() { return this.ite; } public void set_ite(InvalidTopologyException ite) { this.ite = ite; } public void unset_ite() { this.ite = null; } /** Returns true if field ite is set (has been assigned a value) and false otherwise */ public boolean is_set_ite() { return this.ite != null; } public void set_ite_isSet(boolean value) { if (!value) { this.ite = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case E: if (value == null) { unset_e(); } else { set_e((AlreadyAliveException)value); } break; case ITE: if (value == null) { unset_ite(); } else { set_ite((InvalidTopologyException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case E: return get_e(); case ITE: return get_ite(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case E: return is_set_e(); case ITE: return is_set_ite(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof submitTopology_result) return this.equals((submitTopology_result)that); return false; } public boolean equals(submitTopology_result that) { if (that == null) return false; boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } boolean this_present_ite = true && this.is_set_ite(); boolean that_present_ite = true && that.is_set_ite(); if (this_present_ite || that_present_ite) { if (!(this_present_ite && that_present_ite)) return false; if (!this.ite.equals(that.ite)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); boolean present_ite = true && (is_set_ite()); builder.append(present_ite); if (present_ite) builder.append(ite); return builder.toHashCode(); } public int compareTo(submitTopology_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; submitTopology_result typedOther = (submitTopology_result)other; lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); if (lastComparison != 0) { return lastComparison; } if (is_set_ite()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.ite, typedOther.ite); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new AlreadyAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // ITE if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.ite = new InvalidTopologyException(); this.ite.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } else if (this.is_set_ite()) { oprot.writeFieldBegin(ITE_FIELD_DESC); this.ite.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("submitTopology_result("); boolean first = true; sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; if (!first) sb.append(", "); sb.append("ite:"); if (this.ite == null) { sb.append("null"); } else { sb.append(this.ite); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class submitTopologyWithOpts_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopologyWithOpts_args"); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField UPLOADED_JAR_LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("uploadedJarLocation", org.apache.thrift7.protocol.TType.STRING, (short)2); private static final org.apache.thrift7.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift7.protocol.TField("jsonConf", org.apache.thrift7.protocol.TType.STRING, (short)3); private static final org.apache.thrift7.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift7.protocol.TField("topology", org.apache.thrift7.protocol.TType.STRUCT, (short)4); private static final org.apache.thrift7.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift7.protocol.TField("options", org.apache.thrift7.protocol.TType.STRUCT, (short)5); private String name; // required private String uploadedJarLocation; // required private String jsonConf; // required private StormTopology topology; // required private SubmitOptions options; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { NAME((short)1, "name"), UPLOADED_JAR_LOCATION((short)2, "uploadedJarLocation"), JSON_CONF((short)3, "jsonConf"), TOPOLOGY((short)4, "topology"), OPTIONS((short)5, "options"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // NAME return NAME; case 2: // UPLOADED_JAR_LOCATION return UPLOADED_JAR_LOCATION; case 3: // JSON_CONF return JSON_CONF; case 4: // TOPOLOGY return TOPOLOGY; case 5: // OPTIONS return OPTIONS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.UPLOADED_JAR_LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("uploadedJarLocation", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift7.meta_data.FieldMetaData("jsonConf", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift7.meta_data.FieldMetaData("topology", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); tmpMap.put(_Fields.OPTIONS, new org.apache.thrift7.meta_data.FieldMetaData("options", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SubmitOptions.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopologyWithOpts_args.class, metaDataMap); } public submitTopologyWithOpts_args() { } public submitTopologyWithOpts_args( String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) { this(); this.name = name; this.uploadedJarLocation = uploadedJarLocation; this.jsonConf = jsonConf; this.topology = topology; this.options = options; } /** * Performs a deep copy on other. */ public submitTopologyWithOpts_args(submitTopologyWithOpts_args other) { if (other.is_set_name()) { this.name = other.name; } if (other.is_set_uploadedJarLocation()) { this.uploadedJarLocation = other.uploadedJarLocation; } if (other.is_set_jsonConf()) { this.jsonConf = other.jsonConf; } if (other.is_set_topology()) { this.topology = new StormTopology(other.topology); } if (other.is_set_options()) { this.options = new SubmitOptions(other.options); } } public submitTopologyWithOpts_args deepCopy() { return new submitTopologyWithOpts_args(this); } @Override public void clear() { this.name = null; this.uploadedJarLocation = null; this.jsonConf = null; this.topology = null; this.options = null; } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public String get_uploadedJarLocation() { return this.uploadedJarLocation; } public void set_uploadedJarLocation(String uploadedJarLocation) { this.uploadedJarLocation = uploadedJarLocation; } public void unset_uploadedJarLocation() { this.uploadedJarLocation = null; } /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ public boolean is_set_uploadedJarLocation() { return this.uploadedJarLocation != null; } public void set_uploadedJarLocation_isSet(boolean value) { if (!value) { this.uploadedJarLocation = null; } } public String get_jsonConf() { return this.jsonConf; } public void set_jsonConf(String jsonConf) { this.jsonConf = jsonConf; } public void unset_jsonConf() { this.jsonConf = null; } /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ public boolean is_set_jsonConf() { return this.jsonConf != null; } public void set_jsonConf_isSet(boolean value) { if (!value) { this.jsonConf = null; } } public StormTopology get_topology() { return this.topology; } public void set_topology(StormTopology topology) { this.topology = topology; } public void unset_topology() { this.topology = null; } /** Returns true if field topology is set (has been assigned a value) and false otherwise */ public boolean is_set_topology() { return this.topology != null; } public void set_topology_isSet(boolean value) { if (!value) { this.topology = null; } } public SubmitOptions get_options() { return this.options; } public void set_options(SubmitOptions options) { this.options = options; } public void unset_options() { this.options = null; } /** Returns true if field options is set (has been assigned a value) and false otherwise */ public boolean is_set_options() { return this.options != null; } public void set_options_isSet(boolean value) { if (!value) { this.options = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; case UPLOADED_JAR_LOCATION: if (value == null) { unset_uploadedJarLocation(); } else { set_uploadedJarLocation((String)value); } break; case JSON_CONF: if (value == null) { unset_jsonConf(); } else { set_jsonConf((String)value); } break; case TOPOLOGY: if (value == null) { unset_topology(); } else { set_topology((StormTopology)value); } break; case OPTIONS: if (value == null) { unset_options(); } else { set_options((SubmitOptions)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case NAME: return get_name(); case UPLOADED_JAR_LOCATION: return get_uploadedJarLocation(); case JSON_CONF: return get_jsonConf(); case TOPOLOGY: return get_topology(); case OPTIONS: return get_options(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case NAME: return is_set_name(); case UPLOADED_JAR_LOCATION: return is_set_uploadedJarLocation(); case JSON_CONF: return is_set_jsonConf(); case TOPOLOGY: return is_set_topology(); case OPTIONS: return is_set_options(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof submitTopologyWithOpts_args) return this.equals((submitTopologyWithOpts_args)that); return false; } public boolean equals(submitTopologyWithOpts_args that) { if (that == null) return false; boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } boolean this_present_uploadedJarLocation = true && this.is_set_uploadedJarLocation(); boolean that_present_uploadedJarLocation = true && that.is_set_uploadedJarLocation(); if (this_present_uploadedJarLocation || that_present_uploadedJarLocation) { if (!(this_present_uploadedJarLocation && that_present_uploadedJarLocation)) return false; if (!this.uploadedJarLocation.equals(that.uploadedJarLocation)) return false; } boolean this_present_jsonConf = true && this.is_set_jsonConf(); boolean that_present_jsonConf = true && that.is_set_jsonConf(); if (this_present_jsonConf || that_present_jsonConf) { if (!(this_present_jsonConf && that_present_jsonConf)) return false; if (!this.jsonConf.equals(that.jsonConf)) return false; } boolean this_present_topology = true && this.is_set_topology(); boolean that_present_topology = true && that.is_set_topology(); if (this_present_topology || that_present_topology) { if (!(this_present_topology && that_present_topology)) return false; if (!this.topology.equals(that.topology)) return false; } boolean this_present_options = true && this.is_set_options(); boolean that_present_options = true && that.is_set_options(); if (this_present_options || that_present_options) { if (!(this_present_options && that_present_options)) return false; if (!this.options.equals(that.options)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); boolean present_uploadedJarLocation = true && (is_set_uploadedJarLocation()); builder.append(present_uploadedJarLocation); if (present_uploadedJarLocation) builder.append(uploadedJarLocation); boolean present_jsonConf = true && (is_set_jsonConf()); builder.append(present_jsonConf); if (present_jsonConf) builder.append(jsonConf); boolean present_topology = true && (is_set_topology()); builder.append(present_topology); if (present_topology) builder.append(topology); boolean present_options = true && (is_set_options()); builder.append(present_options); if (present_options) builder.append(options); return builder.toHashCode(); } public int compareTo(submitTopologyWithOpts_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; submitTopologyWithOpts_args typedOther = (submitTopologyWithOpts_args)other; lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_uploadedJarLocation()).compareTo(typedOther.is_set_uploadedJarLocation()); if (lastComparison != 0) { return lastComparison; } if (is_set_uploadedJarLocation()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uploadedJarLocation, typedOther.uploadedJarLocation); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(typedOther.is_set_jsonConf()); if (lastComparison != 0) { return lastComparison; } if (is_set_jsonConf()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_topology()).compareTo(typedOther.is_set_topology()); if (lastComparison != 0) { return lastComparison; } if (is_set_topology()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topology, typedOther.topology); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_options()).compareTo(typedOther.is_set_options()); if (lastComparison != 0) { return lastComparison; } if (is_set_options()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.options, typedOther.options); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // UPLOADED_JAR_LOCATION if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.uploadedJarLocation = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // JSON_CONF if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.jsonConf = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // TOPOLOGY if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.topology = new StormTopology(); this.topology.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 5: // OPTIONS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.options = new SubmitOptions(); this.options.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } if (this.uploadedJarLocation != null) { oprot.writeFieldBegin(UPLOADED_JAR_LOCATION_FIELD_DESC); oprot.writeString(this.uploadedJarLocation); oprot.writeFieldEnd(); } if (this.jsonConf != null) { oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); oprot.writeString(this.jsonConf); oprot.writeFieldEnd(); } if (this.topology != null) { oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); this.topology.write(oprot); oprot.writeFieldEnd(); } if (this.options != null) { oprot.writeFieldBegin(OPTIONS_FIELD_DESC); this.options.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("submitTopologyWithOpts_args("); boolean first = true; sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; if (!first) sb.append(", "); sb.append("uploadedJarLocation:"); if (this.uploadedJarLocation == null) { sb.append("null"); } else { sb.append(this.uploadedJarLocation); } first = false; if (!first) sb.append(", "); sb.append("jsonConf:"); if (this.jsonConf == null) { sb.append("null"); } else { sb.append(this.jsonConf); } first = false; if (!first) sb.append(", "); sb.append("topology:"); if (this.topology == null) { sb.append("null"); } else { sb.append(this.topology); } first = false; if (!first) sb.append(", "); sb.append("options:"); if (this.options == null) { sb.append("null"); } else { sb.append(this.options); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class submitTopologyWithOpts_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopologyWithOpts_result"); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField ITE_FIELD_DESC = new org.apache.thrift7.protocol.TField("ite", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private AlreadyAliveException e; // required private InvalidTopologyException ite; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { E((short)1, "e"), ITE((short)2, "ite"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; case 2: // ITE return ITE; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); tmpMap.put(_Fields.ITE, new org.apache.thrift7.meta_data.FieldMetaData("ite", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopologyWithOpts_result.class, metaDataMap); } public submitTopologyWithOpts_result() { } public submitTopologyWithOpts_result( AlreadyAliveException e, InvalidTopologyException ite) { this(); this.e = e; this.ite = ite; } /** * Performs a deep copy on other. */ public submitTopologyWithOpts_result(submitTopologyWithOpts_result other) { if (other.is_set_e()) { this.e = new AlreadyAliveException(other.e); } if (other.is_set_ite()) { this.ite = new InvalidTopologyException(other.ite); } } public submitTopologyWithOpts_result deepCopy() { return new submitTopologyWithOpts_result(this); } @Override public void clear() { this.e = null; this.ite = null; } public AlreadyAliveException get_e() { return this.e; } public void set_e(AlreadyAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public InvalidTopologyException get_ite() { return this.ite; } public void set_ite(InvalidTopologyException ite) { this.ite = ite; } public void unset_ite() { this.ite = null; } /** Returns true if field ite is set (has been assigned a value) and false otherwise */ public boolean is_set_ite() { return this.ite != null; } public void set_ite_isSet(boolean value) { if (!value) { this.ite = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case E: if (value == null) { unset_e(); } else { set_e((AlreadyAliveException)value); } break; case ITE: if (value == null) { unset_ite(); } else { set_ite((InvalidTopologyException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case E: return get_e(); case ITE: return get_ite(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case E: return is_set_e(); case ITE: return is_set_ite(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof submitTopologyWithOpts_result) return this.equals((submitTopologyWithOpts_result)that); return false; } public boolean equals(submitTopologyWithOpts_result that) { if (that == null) return false; boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } boolean this_present_ite = true && this.is_set_ite(); boolean that_present_ite = true && that.is_set_ite(); if (this_present_ite || that_present_ite) { if (!(this_present_ite && that_present_ite)) return false; if (!this.ite.equals(that.ite)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); boolean present_ite = true && (is_set_ite()); builder.append(present_ite); if (present_ite) builder.append(ite); return builder.toHashCode(); } public int compareTo(submitTopologyWithOpts_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; submitTopologyWithOpts_result typedOther = (submitTopologyWithOpts_result)other; lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); if (lastComparison != 0) { return lastComparison; } if (is_set_ite()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.ite, typedOther.ite); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new AlreadyAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // ITE if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.ite = new InvalidTopologyException(); this.ite.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } else if (this.is_set_ite()) { oprot.writeFieldBegin(ITE_FIELD_DESC); this.ite.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("submitTopologyWithOpts_result("); boolean first = true; sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; if (!first) sb.append(", "); sb.append("ite:"); if (this.ite == null) { sb.append("null"); } else { sb.append(this.ite); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class killTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopology_args"); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); private String name; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { NAME((short)1, "name"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // NAME return NAME; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopology_args.class, metaDataMap); } public killTopology_args() { } public killTopology_args( String name) { this(); this.name = name; } /** * Performs a deep copy on other. */ public killTopology_args(killTopology_args other) { if (other.is_set_name()) { this.name = other.name; } } public killTopology_args deepCopy() { return new killTopology_args(this); } @Override public void clear() { this.name = null; } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case NAME: return get_name(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case NAME: return is_set_name(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof killTopology_args) return this.equals((killTopology_args)that); return false; } public boolean equals(killTopology_args that) { if (that == null) return false; boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); return builder.toHashCode(); } public int compareTo(killTopology_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; killTopology_args typedOther = (killTopology_args)other; lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("killTopology_args("); boolean first = true; sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class killTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopology_result"); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopology_result.class, metaDataMap); } public killTopology_result() { } public killTopology_result( NotAliveException e) { this(); this.e = e; } /** * Performs a deep copy on other. */ public killTopology_result(killTopology_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public killTopology_result deepCopy() { return new killTopology_result(this); } @Override public void clear() { this.e = null; } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof killTopology_result) return this.equals((killTopology_result)that); return false; } public boolean equals(killTopology_result that) { if (that == null) return false; boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(killTopology_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; killTopology_result typedOther = (killTopology_result)other; lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("killTopology_result("); boolean first = true; sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class killTopologyWithOpts_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopologyWithOpts_args"); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift7.protocol.TField("options", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private String name; // required private KillOptions options; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { NAME((short)1, "name"), OPTIONS((short)2, "options"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // NAME return NAME; case 2: // OPTIONS return OPTIONS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.OPTIONS, new org.apache.thrift7.meta_data.FieldMetaData("options", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, KillOptions.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopologyWithOpts_args.class, metaDataMap); } public killTopologyWithOpts_args() { } public killTopologyWithOpts_args( String name, KillOptions options) { this(); this.name = name; this.options = options; } /** * Performs a deep copy on other. */ public killTopologyWithOpts_args(killTopologyWithOpts_args other) { if (other.is_set_name()) { this.name = other.name; } if (other.is_set_options()) { this.options = new KillOptions(other.options); } } public killTopologyWithOpts_args deepCopy() { return new killTopologyWithOpts_args(this); } @Override public void clear() { this.name = null; this.options = null; } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public KillOptions get_options() { return this.options; } public void set_options(KillOptions options) { this.options = options; } public void unset_options() { this.options = null; } /** Returns true if field options is set (has been assigned a value) and false otherwise */ public boolean is_set_options() { return this.options != null; } public void set_options_isSet(boolean value) { if (!value) { this.options = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; case OPTIONS: if (value == null) { unset_options(); } else { set_options((KillOptions)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case NAME: return get_name(); case OPTIONS: return get_options(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case NAME: return is_set_name(); case OPTIONS: return is_set_options(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof killTopologyWithOpts_args) return this.equals((killTopologyWithOpts_args)that); return false; } public boolean equals(killTopologyWithOpts_args that) { if (that == null) return false; boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } boolean this_present_options = true && this.is_set_options(); boolean that_present_options = true && that.is_set_options(); if (this_present_options || that_present_options) { if (!(this_present_options && that_present_options)) return false; if (!this.options.equals(that.options)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); boolean present_options = true && (is_set_options()); builder.append(present_options); if (present_options) builder.append(options); return builder.toHashCode(); } public int compareTo(killTopologyWithOpts_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; killTopologyWithOpts_args typedOther = (killTopologyWithOpts_args)other; lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_options()).compareTo(typedOther.is_set_options()); if (lastComparison != 0) { return lastComparison; } if (is_set_options()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.options, typedOther.options); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // OPTIONS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.options = new KillOptions(); this.options.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } if (this.options != null) { oprot.writeFieldBegin(OPTIONS_FIELD_DESC); this.options.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("killTopologyWithOpts_args("); boolean first = true; sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; if (!first) sb.append(", "); sb.append("options:"); if (this.options == null) { sb.append("null"); } else { sb.append(this.options); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class killTopologyWithOpts_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopologyWithOpts_result"); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopologyWithOpts_result.class, metaDataMap); } public killTopologyWithOpts_result() { } public killTopologyWithOpts_result( NotAliveException e) { this(); this.e = e; } /** * Performs a deep copy on other. */ public killTopologyWithOpts_result(killTopologyWithOpts_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public killTopologyWithOpts_result deepCopy() { return new killTopologyWithOpts_result(this); } @Override public void clear() { this.e = null; } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof killTopologyWithOpts_result) return this.equals((killTopologyWithOpts_result)that); return false; } public boolean equals(killTopologyWithOpts_result that) { if (that == null) return false; boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(killTopologyWithOpts_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; killTopologyWithOpts_result typedOther = (killTopologyWithOpts_result)other; lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("killTopologyWithOpts_result("); boolean first = true; sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class activate_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("activate_args"); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); private String name; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { NAME((short)1, "name"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // NAME return NAME; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(activate_args.class, metaDataMap); } public activate_args() { } public activate_args( String name) { this(); this.name = name; } /** * Performs a deep copy on other. */ public activate_args(activate_args other) { if (other.is_set_name()) { this.name = other.name; } } public activate_args deepCopy() { return new activate_args(this); } @Override public void clear() { this.name = null; } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case NAME: return get_name(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case NAME: return is_set_name(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof activate_args) return this.equals((activate_args)that); return false; } public boolean equals(activate_args that) { if (that == null) return false; boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); return builder.toHashCode(); } public int compareTo(activate_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; activate_args typedOther = (activate_args)other; lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("activate_args("); boolean first = true; sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class activate_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("activate_result"); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(activate_result.class, metaDataMap); } public activate_result() { } public activate_result( NotAliveException e) { this(); this.e = e; } /** * Performs a deep copy on other. */ public activate_result(activate_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public activate_result deepCopy() { return new activate_result(this); } @Override public void clear() { this.e = null; } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof activate_result) return this.equals((activate_result)that); return false; } public boolean equals(activate_result that) { if (that == null) return false; boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(activate_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; activate_result typedOther = (activate_result)other; lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("activate_result("); boolean first = true; sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class deactivate_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("deactivate_args"); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); private String name; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { NAME((short)1, "name"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // NAME return NAME; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(deactivate_args.class, metaDataMap); } public deactivate_args() { } public deactivate_args( String name) { this(); this.name = name; } /** * Performs a deep copy on other. */ public deactivate_args(deactivate_args other) { if (other.is_set_name()) { this.name = other.name; } } public deactivate_args deepCopy() { return new deactivate_args(this); } @Override public void clear() { this.name = null; } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case NAME: return get_name(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case NAME: return is_set_name(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof deactivate_args) return this.equals((deactivate_args)that); return false; } public boolean equals(deactivate_args that) { if (that == null) return false; boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); return builder.toHashCode(); } public int compareTo(deactivate_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; deactivate_args typedOther = (deactivate_args)other; lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("deactivate_args("); boolean first = true; sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class deactivate_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("deactivate_result"); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(deactivate_result.class, metaDataMap); } public deactivate_result() { } public deactivate_result( NotAliveException e) { this(); this.e = e; } /** * Performs a deep copy on other. */ public deactivate_result(deactivate_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public deactivate_result deepCopy() { return new deactivate_result(this); } @Override public void clear() { this.e = null; } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof deactivate_result) return this.equals((deactivate_result)that); return false; } public boolean equals(deactivate_result that) { if (that == null) return false; boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(deactivate_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; deactivate_result typedOther = (deactivate_result)other; lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("deactivate_result("); boolean first = true; sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class rebalance_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("rebalance_args"); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift7.protocol.TField("options", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private String name; // required private RebalanceOptions options; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { NAME((short)1, "name"), OPTIONS((short)2, "options"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // NAME return NAME; case 2: // OPTIONS return OPTIONS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.OPTIONS, new org.apache.thrift7.meta_data.FieldMetaData("options", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, RebalanceOptions.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(rebalance_args.class, metaDataMap); } public rebalance_args() { } public rebalance_args( String name, RebalanceOptions options) { this(); this.name = name; this.options = options; } /** * Performs a deep copy on other. */ public rebalance_args(rebalance_args other) { if (other.is_set_name()) { this.name = other.name; } if (other.is_set_options()) { this.options = new RebalanceOptions(other.options); } } public rebalance_args deepCopy() { return new rebalance_args(this); } @Override public void clear() { this.name = null; this.options = null; } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public RebalanceOptions get_options() { return this.options; } public void set_options(RebalanceOptions options) { this.options = options; } public void unset_options() { this.options = null; } /** Returns true if field options is set (has been assigned a value) and false otherwise */ public boolean is_set_options() { return this.options != null; } public void set_options_isSet(boolean value) { if (!value) { this.options = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; case OPTIONS: if (value == null) { unset_options(); } else { set_options((RebalanceOptions)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case NAME: return get_name(); case OPTIONS: return get_options(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case NAME: return is_set_name(); case OPTIONS: return is_set_options(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof rebalance_args) return this.equals((rebalance_args)that); return false; } public boolean equals(rebalance_args that) { if (that == null) return false; boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } boolean this_present_options = true && this.is_set_options(); boolean that_present_options = true && that.is_set_options(); if (this_present_options || that_present_options) { if (!(this_present_options && that_present_options)) return false; if (!this.options.equals(that.options)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); boolean present_options = true && (is_set_options()); builder.append(present_options); if (present_options) builder.append(options); return builder.toHashCode(); } public int compareTo(rebalance_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; rebalance_args typedOther = (rebalance_args)other; lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_options()).compareTo(typedOther.is_set_options()); if (lastComparison != 0) { return lastComparison; } if (is_set_options()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.options, typedOther.options); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // OPTIONS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.options = new RebalanceOptions(); this.options.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } if (this.options != null) { oprot.writeFieldBegin(OPTIONS_FIELD_DESC); this.options.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("rebalance_args("); boolean first = true; sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; if (!first) sb.append(", "); sb.append("options:"); if (this.options == null) { sb.append("null"); } else { sb.append(this.options); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class rebalance_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("rebalance_result"); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField ITE_FIELD_DESC = new org.apache.thrift7.protocol.TField("ite", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private NotAliveException e; // required private InvalidTopologyException ite; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { E((short)1, "e"), ITE((short)2, "ite"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; case 2: // ITE return ITE; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); tmpMap.put(_Fields.ITE, new org.apache.thrift7.meta_data.FieldMetaData("ite", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(rebalance_result.class, metaDataMap); } public rebalance_result() { } public rebalance_result( NotAliveException e, InvalidTopologyException ite) { this(); this.e = e; this.ite = ite; } /** * Performs a deep copy on other. */ public rebalance_result(rebalance_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } if (other.is_set_ite()) { this.ite = new InvalidTopologyException(other.ite); } } public rebalance_result deepCopy() { return new rebalance_result(this); } @Override public void clear() { this.e = null; this.ite = null; } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public InvalidTopologyException get_ite() { return this.ite; } public void set_ite(InvalidTopologyException ite) { this.ite = ite; } public void unset_ite() { this.ite = null; } /** Returns true if field ite is set (has been assigned a value) and false otherwise */ public boolean is_set_ite() { return this.ite != null; } public void set_ite_isSet(boolean value) { if (!value) { this.ite = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; case ITE: if (value == null) { unset_ite(); } else { set_ite((InvalidTopologyException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case E: return get_e(); case ITE: return get_ite(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case E: return is_set_e(); case ITE: return is_set_ite(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof rebalance_result) return this.equals((rebalance_result)that); return false; } public boolean equals(rebalance_result that) { if (that == null) return false; boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } boolean this_present_ite = true && this.is_set_ite(); boolean that_present_ite = true && that.is_set_ite(); if (this_present_ite || that_present_ite) { if (!(this_present_ite && that_present_ite)) return false; if (!this.ite.equals(that.ite)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); boolean present_ite = true && (is_set_ite()); builder.append(present_ite); if (present_ite) builder.append(ite); return builder.toHashCode(); } public int compareTo(rebalance_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; rebalance_result typedOther = (rebalance_result)other; lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); if (lastComparison != 0) { return lastComparison; } if (is_set_ite()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.ite, typedOther.ite); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // ITE if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.ite = new InvalidTopologyException(); this.ite.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } else if (this.is_set_ite()) { oprot.writeFieldBegin(ITE_FIELD_DESC); this.ite.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("rebalance_result("); boolean first = true; sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; if (!first) sb.append(", "); sb.append("ite:"); if (this.ite == null) { sb.append("null"); } else { sb.append(this.ite); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class beginFileUpload_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileUpload_args"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileUpload_args.class, metaDataMap); } public beginFileUpload_args() { } /** * Performs a deep copy on other. */ public beginFileUpload_args(beginFileUpload_args other) { } public beginFileUpload_args deepCopy() { return new beginFileUpload_args(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof beginFileUpload_args) return this.equals((beginFileUpload_args)that); return false; } public boolean equals(beginFileUpload_args that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(beginFileUpload_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; beginFileUpload_args typedOther = (beginFileUpload_args)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("beginFileUpload_args("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class beginFileUpload_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileUpload_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); private String success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileUpload_result.class, metaDataMap); } public beginFileUpload_result() { } public beginFileUpload_result( String success) { this(); this.success = success; } /** * Performs a deep copy on other. */ public beginFileUpload_result(beginFileUpload_result other) { if (other.is_set_success()) { this.success = other.success; } } public beginFileUpload_result deepCopy() { return new beginFileUpload_result(this); } @Override public void clear() { this.success = null; } public String get_success() { return this.success; } public void set_success(String success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof beginFileUpload_result) return this.equals((beginFileUpload_result)that); return false; } public boolean equals(beginFileUpload_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); return builder.toHashCode(); } public int compareTo(beginFileUpload_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; beginFileUpload_result typedOther = (beginFileUpload_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.success = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("beginFileUpload_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class uploadChunk_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("uploadChunk_args"); private static final org.apache.thrift7.protocol.TField LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("location", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField CHUNK_FIELD_DESC = new org.apache.thrift7.protocol.TField("chunk", org.apache.thrift7.protocol.TType.STRING, (short)2); private String location; // required private ByteBuffer chunk; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { LOCATION((short)1, "location"), CHUNK((short)2, "chunk"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // LOCATION return LOCATION; case 2: // CHUNK return CHUNK; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("location", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.CHUNK, new org.apache.thrift7.meta_data.FieldMetaData("chunk", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(uploadChunk_args.class, metaDataMap); } public uploadChunk_args() { } public uploadChunk_args( String location, ByteBuffer chunk) { this(); this.location = location; this.chunk = chunk; } /** * Performs a deep copy on other. */ public uploadChunk_args(uploadChunk_args other) { if (other.is_set_location()) { this.location = other.location; } if (other.is_set_chunk()) { this.chunk = org.apache.thrift7.TBaseHelper.copyBinary(other.chunk); ; } } public uploadChunk_args deepCopy() { return new uploadChunk_args(this); } @Override public void clear() { this.location = null; this.chunk = null; } public String get_location() { return this.location; } public void set_location(String location) { this.location = location; } public void unset_location() { this.location = null; } /** Returns true if field location is set (has been assigned a value) and false otherwise */ public boolean is_set_location() { return this.location != null; } public void set_location_isSet(boolean value) { if (!value) { this.location = null; } } public byte[] get_chunk() { set_chunk(org.apache.thrift7.TBaseHelper.rightSize(chunk)); return chunk == null ? null : chunk.array(); } public ByteBuffer buffer_for_chunk() { return chunk; } public void set_chunk(byte[] chunk) { set_chunk(chunk == null ? (ByteBuffer)null : ByteBuffer.wrap(chunk)); } public void set_chunk(ByteBuffer chunk) { this.chunk = chunk; } public void unset_chunk() { this.chunk = null; } /** Returns true if field chunk is set (has been assigned a value) and false otherwise */ public boolean is_set_chunk() { return this.chunk != null; } public void set_chunk_isSet(boolean value) { if (!value) { this.chunk = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case LOCATION: if (value == null) { unset_location(); } else { set_location((String)value); } break; case CHUNK: if (value == null) { unset_chunk(); } else { set_chunk((ByteBuffer)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case LOCATION: return get_location(); case CHUNK: return get_chunk(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case LOCATION: return is_set_location(); case CHUNK: return is_set_chunk(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof uploadChunk_args) return this.equals((uploadChunk_args)that); return false; } public boolean equals(uploadChunk_args that) { if (that == null) return false; boolean this_present_location = true && this.is_set_location(); boolean that_present_location = true && that.is_set_location(); if (this_present_location || that_present_location) { if (!(this_present_location && that_present_location)) return false; if (!this.location.equals(that.location)) return false; } boolean this_present_chunk = true && this.is_set_chunk(); boolean that_present_chunk = true && that.is_set_chunk(); if (this_present_chunk || that_present_chunk) { if (!(this_present_chunk && that_present_chunk)) return false; if (!this.chunk.equals(that.chunk)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_location = true && (is_set_location()); builder.append(present_location); if (present_location) builder.append(location); boolean present_chunk = true && (is_set_chunk()); builder.append(present_chunk); if (present_chunk) builder.append(chunk); return builder.toHashCode(); } public int compareTo(uploadChunk_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; uploadChunk_args typedOther = (uploadChunk_args)other; lastComparison = Boolean.valueOf(is_set_location()).compareTo(typedOther.is_set_location()); if (lastComparison != 0) { return lastComparison; } if (is_set_location()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.location, typedOther.location); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_chunk()).compareTo(typedOther.is_set_chunk()); if (lastComparison != 0) { return lastComparison; } if (is_set_chunk()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.chunk, typedOther.chunk); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // LOCATION if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.location = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // CHUNK if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.chunk = iprot.readBinary(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.location != null) { oprot.writeFieldBegin(LOCATION_FIELD_DESC); oprot.writeString(this.location); oprot.writeFieldEnd(); } if (this.chunk != null) { oprot.writeFieldBegin(CHUNK_FIELD_DESC); oprot.writeBinary(this.chunk); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("uploadChunk_args("); boolean first = true; sb.append("location:"); if (this.location == null) { sb.append("null"); } else { sb.append(this.location); } first = false; if (!first) sb.append(", "); sb.append("chunk:"); if (this.chunk == null) { sb.append("null"); } else { org.apache.thrift7.TBaseHelper.toString(this.chunk, sb); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class uploadChunk_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("uploadChunk_result"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(uploadChunk_result.class, metaDataMap); } public uploadChunk_result() { } /** * Performs a deep copy on other. */ public uploadChunk_result(uploadChunk_result other) { } public uploadChunk_result deepCopy() { return new uploadChunk_result(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof uploadChunk_result) return this.equals((uploadChunk_result)that); return false; } public boolean equals(uploadChunk_result that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(uploadChunk_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; uploadChunk_result typedOther = (uploadChunk_result)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("uploadChunk_result("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class finishFileUpload_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("finishFileUpload_args"); private static final org.apache.thrift7.protocol.TField LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("location", org.apache.thrift7.protocol.TType.STRING, (short)1); private String location; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { LOCATION((short)1, "location"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // LOCATION return LOCATION; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("location", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(finishFileUpload_args.class, metaDataMap); } public finishFileUpload_args() { } public finishFileUpload_args( String location) { this(); this.location = location; } /** * Performs a deep copy on other. */ public finishFileUpload_args(finishFileUpload_args other) { if (other.is_set_location()) { this.location = other.location; } } public finishFileUpload_args deepCopy() { return new finishFileUpload_args(this); } @Override public void clear() { this.location = null; } public String get_location() { return this.location; } public void set_location(String location) { this.location = location; } public void unset_location() { this.location = null; } /** Returns true if field location is set (has been assigned a value) and false otherwise */ public boolean is_set_location() { return this.location != null; } public void set_location_isSet(boolean value) { if (!value) { this.location = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case LOCATION: if (value == null) { unset_location(); } else { set_location((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case LOCATION: return get_location(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case LOCATION: return is_set_location(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof finishFileUpload_args) return this.equals((finishFileUpload_args)that); return false; } public boolean equals(finishFileUpload_args that) { if (that == null) return false; boolean this_present_location = true && this.is_set_location(); boolean that_present_location = true && that.is_set_location(); if (this_present_location || that_present_location) { if (!(this_present_location && that_present_location)) return false; if (!this.location.equals(that.location)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_location = true && (is_set_location()); builder.append(present_location); if (present_location) builder.append(location); return builder.toHashCode(); } public int compareTo(finishFileUpload_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; finishFileUpload_args typedOther = (finishFileUpload_args)other; lastComparison = Boolean.valueOf(is_set_location()).compareTo(typedOther.is_set_location()); if (lastComparison != 0) { return lastComparison; } if (is_set_location()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.location, typedOther.location); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // LOCATION if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.location = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.location != null) { oprot.writeFieldBegin(LOCATION_FIELD_DESC); oprot.writeString(this.location); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("finishFileUpload_args("); boolean first = true; sb.append("location:"); if (this.location == null) { sb.append("null"); } else { sb.append(this.location); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class finishFileUpload_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("finishFileUpload_result"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(finishFileUpload_result.class, metaDataMap); } public finishFileUpload_result() { } /** * Performs a deep copy on other. */ public finishFileUpload_result(finishFileUpload_result other) { } public finishFileUpload_result deepCopy() { return new finishFileUpload_result(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof finishFileUpload_result) return this.equals((finishFileUpload_result)that); return false; } public boolean equals(finishFileUpload_result that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(finishFileUpload_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; finishFileUpload_result typedOther = (finishFileUpload_result)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("finishFileUpload_result("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class beginFileDownload_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileDownload_args"); private static final org.apache.thrift7.protocol.TField FILE_FIELD_DESC = new org.apache.thrift7.protocol.TField("file", org.apache.thrift7.protocol.TType.STRING, (short)1); private String file; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { FILE((short)1, "file"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // FILE return FILE; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.FILE, new org.apache.thrift7.meta_data.FieldMetaData("file", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileDownload_args.class, metaDataMap); } public beginFileDownload_args() { } public beginFileDownload_args( String file) { this(); this.file = file; } /** * Performs a deep copy on other. */ public beginFileDownload_args(beginFileDownload_args other) { if (other.is_set_file()) { this.file = other.file; } } public beginFileDownload_args deepCopy() { return new beginFileDownload_args(this); } @Override public void clear() { this.file = null; } public String get_file() { return this.file; } public void set_file(String file) { this.file = file; } public void unset_file() { this.file = null; } /** Returns true if field file is set (has been assigned a value) and false otherwise */ public boolean is_set_file() { return this.file != null; } public void set_file_isSet(boolean value) { if (!value) { this.file = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case FILE: if (value == null) { unset_file(); } else { set_file((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case FILE: return get_file(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case FILE: return is_set_file(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof beginFileDownload_args) return this.equals((beginFileDownload_args)that); return false; } public boolean equals(beginFileDownload_args that) { if (that == null) return false; boolean this_present_file = true && this.is_set_file(); boolean that_present_file = true && that.is_set_file(); if (this_present_file || that_present_file) { if (!(this_present_file && that_present_file)) return false; if (!this.file.equals(that.file)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_file = true && (is_set_file()); builder.append(present_file); if (present_file) builder.append(file); return builder.toHashCode(); } public int compareTo(beginFileDownload_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; beginFileDownload_args typedOther = (beginFileDownload_args)other; lastComparison = Boolean.valueOf(is_set_file()).compareTo(typedOther.is_set_file()); if (lastComparison != 0) { return lastComparison; } if (is_set_file()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.file, typedOther.file); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // FILE if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.file = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.file != null) { oprot.writeFieldBegin(FILE_FIELD_DESC); oprot.writeString(this.file); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("beginFileDownload_args("); boolean first = true; sb.append("file:"); if (this.file == null) { sb.append("null"); } else { sb.append(this.file); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class beginFileDownload_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileDownload_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); private String success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileDownload_result.class, metaDataMap); } public beginFileDownload_result() { } public beginFileDownload_result( String success) { this(); this.success = success; } /** * Performs a deep copy on other. */ public beginFileDownload_result(beginFileDownload_result other) { if (other.is_set_success()) { this.success = other.success; } } public beginFileDownload_result deepCopy() { return new beginFileDownload_result(this); } @Override public void clear() { this.success = null; } public String get_success() { return this.success; } public void set_success(String success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof beginFileDownload_result) return this.equals((beginFileDownload_result)that); return false; } public boolean equals(beginFileDownload_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); return builder.toHashCode(); } public int compareTo(beginFileDownload_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; beginFileDownload_result typedOther = (beginFileDownload_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.success = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("beginFileDownload_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class downloadChunk_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("downloadChunk_args"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private String id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(downloadChunk_args.class, metaDataMap); } public downloadChunk_args() { } public downloadChunk_args( String id) { this(); this.id = id; } /** * Performs a deep copy on other. */ public downloadChunk_args(downloadChunk_args other) { if (other.is_set_id()) { this.id = other.id; } } public downloadChunk_args deepCopy() { return new downloadChunk_args(this); } @Override public void clear() { this.id = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof downloadChunk_args) return this.equals((downloadChunk_args)that); return false; } public boolean equals(downloadChunk_args that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); return builder.toHashCode(); } public int compareTo(downloadChunk_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; downloadChunk_args typedOther = (downloadChunk_args)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("downloadChunk_args("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class downloadChunk_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("downloadChunk_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); private ByteBuffer success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(downloadChunk_result.class, metaDataMap); } public downloadChunk_result() { } public downloadChunk_result( ByteBuffer success) { this(); this.success = success; } /** * Performs a deep copy on other. */ public downloadChunk_result(downloadChunk_result other) { if (other.is_set_success()) { this.success = org.apache.thrift7.TBaseHelper.copyBinary(other.success); ; } } public downloadChunk_result deepCopy() { return new downloadChunk_result(this); } @Override public void clear() { this.success = null; } public byte[] get_success() { set_success(org.apache.thrift7.TBaseHelper.rightSize(success)); return success == null ? null : success.array(); } public ByteBuffer buffer_for_success() { return success; } public void set_success(byte[] success) { set_success(success == null ? (ByteBuffer)null : ByteBuffer.wrap(success)); } public void set_success(ByteBuffer success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((ByteBuffer)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof downloadChunk_result) return this.equals((downloadChunk_result)that); return false; } public boolean equals(downloadChunk_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); return builder.toHashCode(); } public int compareTo(downloadChunk_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; downloadChunk_result typedOther = (downloadChunk_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.success = iprot.readBinary(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeBinary(this.success); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("downloadChunk_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { org.apache.thrift7.TBaseHelper.toString(this.success, sb); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getNimbusConf_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getNimbusConf_args"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getNimbusConf_args.class, metaDataMap); } public getNimbusConf_args() { } /** * Performs a deep copy on other. */ public getNimbusConf_args(getNimbusConf_args other) { } public getNimbusConf_args deepCopy() { return new getNimbusConf_args(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getNimbusConf_args) return this.equals((getNimbusConf_args)that); return false; } public boolean equals(getNimbusConf_args that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(getNimbusConf_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getNimbusConf_args typedOther = (getNimbusConf_args)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getNimbusConf_args("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getNimbusConf_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getNimbusConf_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); private String success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getNimbusConf_result.class, metaDataMap); } public getNimbusConf_result() { } public getNimbusConf_result( String success) { this(); this.success = success; } /** * Performs a deep copy on other. */ public getNimbusConf_result(getNimbusConf_result other) { if (other.is_set_success()) { this.success = other.success; } } public getNimbusConf_result deepCopy() { return new getNimbusConf_result(this); } @Override public void clear() { this.success = null; } public String get_success() { return this.success; } public void set_success(String success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getNimbusConf_result) return this.equals((getNimbusConf_result)that); return false; } public boolean equals(getNimbusConf_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); return builder.toHashCode(); } public int compareTo(getNimbusConf_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getNimbusConf_result typedOther = (getNimbusConf_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.success = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getNimbusConf_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getClusterInfo_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getClusterInfo_args"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getClusterInfo_args.class, metaDataMap); } public getClusterInfo_args() { } /** * Performs a deep copy on other. */ public getClusterInfo_args(getClusterInfo_args other) { } public getClusterInfo_args deepCopy() { return new getClusterInfo_args(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getClusterInfo_args) return this.equals((getClusterInfo_args)that); return false; } public boolean equals(getClusterInfo_args that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(getClusterInfo_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getClusterInfo_args typedOther = (getClusterInfo_args)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getClusterInfo_args("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getClusterInfo_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getClusterInfo_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); private ClusterSummary success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ClusterSummary.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getClusterInfo_result.class, metaDataMap); } public getClusterInfo_result() { } public getClusterInfo_result( ClusterSummary success) { this(); this.success = success; } /** * Performs a deep copy on other. */ public getClusterInfo_result(getClusterInfo_result other) { if (other.is_set_success()) { this.success = new ClusterSummary(other.success); } } public getClusterInfo_result deepCopy() { return new getClusterInfo_result(this); } @Override public void clear() { this.success = null; } public ClusterSummary get_success() { return this.success; } public void set_success(ClusterSummary success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((ClusterSummary)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getClusterInfo_result) return this.equals((getClusterInfo_result)that); return false; } public boolean equals(getClusterInfo_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); return builder.toHashCode(); } public int compareTo(getClusterInfo_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getClusterInfo_result typedOther = (getClusterInfo_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.success = new ClusterSummary(); this.success.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); this.success.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getClusterInfo_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getTopologyInfo_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyInfo_args"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private String id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfo_args.class, metaDataMap); } public getTopologyInfo_args() { } public getTopologyInfo_args( String id) { this(); this.id = id; } /** * Performs a deep copy on other. */ public getTopologyInfo_args(getTopologyInfo_args other) { if (other.is_set_id()) { this.id = other.id; } } public getTopologyInfo_args deepCopy() { return new getTopologyInfo_args(this); } @Override public void clear() { this.id = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getTopologyInfo_args) return this.equals((getTopologyInfo_args)that); return false; } public boolean equals(getTopologyInfo_args that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); return builder.toHashCode(); } public int compareTo(getTopologyInfo_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getTopologyInfo_args typedOther = (getTopologyInfo_args)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getTopologyInfo_args("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getTopologyInfo_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyInfo_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private TopologyInfo success; // required private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"), E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TopologyInfo.class))); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfo_result.class, metaDataMap); } public getTopologyInfo_result() { } public getTopologyInfo_result( TopologyInfo success, NotAliveException e) { this(); this.success = success; this.e = e; } /** * Performs a deep copy on other. */ public getTopologyInfo_result(getTopologyInfo_result other) { if (other.is_set_success()) { this.success = new TopologyInfo(other.success); } if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public getTopologyInfo_result deepCopy() { return new getTopologyInfo_result(this); } @Override public void clear() { this.success = null; this.e = null; } public TopologyInfo get_success() { return this.success; } public void set_success(TopologyInfo success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((TopologyInfo)value); } break; case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getTopologyInfo_result) return this.equals((getTopologyInfo_result)that); return false; } public boolean equals(getTopologyInfo_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(getTopologyInfo_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getTopologyInfo_result typedOther = (getTopologyInfo_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.success = new TopologyInfo(); this.success.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); this.success.write(oprot); oprot.writeFieldEnd(); } else if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getTopologyInfo_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; if (!first) sb.append(", "); sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getTopologyConf_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyConf_args"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private String id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_args.class, metaDataMap); } public getTopologyConf_args() { } public getTopologyConf_args( String id) { this(); this.id = id; } /** * Performs a deep copy on other. */ public getTopologyConf_args(getTopologyConf_args other) { if (other.is_set_id()) { this.id = other.id; } } public getTopologyConf_args deepCopy() { return new getTopologyConf_args(this); } @Override public void clear() { this.id = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getTopologyConf_args) return this.equals((getTopologyConf_args)that); return false; } public boolean equals(getTopologyConf_args that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); return builder.toHashCode(); } public int compareTo(getTopologyConf_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getTopologyConf_args typedOther = (getTopologyConf_args)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getTopologyConf_args("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getTopologyConf_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyConf_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private String success; // required private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"), E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_result.class, metaDataMap); } public getTopologyConf_result() { } public getTopologyConf_result( String success, NotAliveException e) { this(); this.success = success; this.e = e; } /** * Performs a deep copy on other. */ public getTopologyConf_result(getTopologyConf_result other) { if (other.is_set_success()) { this.success = other.success; } if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public getTopologyConf_result deepCopy() { return new getTopologyConf_result(this); } @Override public void clear() { this.success = null; this.e = null; } public String get_success() { return this.success; } public void set_success(String success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((String)value); } break; case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getTopologyConf_result) return this.equals((getTopologyConf_result)that); return false; } public boolean equals(getTopologyConf_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(getTopologyConf_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getTopologyConf_result typedOther = (getTopologyConf_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.success = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); } else if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getTopologyConf_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; if (!first) sb.append(", "); sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopology_args"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private String id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_args.class, metaDataMap); } public getTopology_args() { } public getTopology_args( String id) { this(); this.id = id; } /** * Performs a deep copy on other. */ public getTopology_args(getTopology_args other) { if (other.is_set_id()) { this.id = other.id; } } public getTopology_args deepCopy() { return new getTopology_args(this); } @Override public void clear() { this.id = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getTopology_args) return this.equals((getTopology_args)that); return false; } public boolean equals(getTopology_args that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); return builder.toHashCode(); } public int compareTo(getTopology_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getTopology_args typedOther = (getTopology_args)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getTopology_args("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopology_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private StormTopology success; // required private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"), E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_result.class, metaDataMap); } public getTopology_result() { } public getTopology_result( StormTopology success, NotAliveException e) { this(); this.success = success; this.e = e; } /** * Performs a deep copy on other. */ public getTopology_result(getTopology_result other) { if (other.is_set_success()) { this.success = new StormTopology(other.success); } if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public getTopology_result deepCopy() { return new getTopology_result(this); } @Override public void clear() { this.success = null; this.e = null; } public StormTopology get_success() { return this.success; } public void set_success(StormTopology success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((StormTopology)value); } break; case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getTopology_result) return this.equals((getTopology_result)that); return false; } public boolean equals(getTopology_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(getTopology_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getTopology_result typedOther = (getTopology_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.success = new StormTopology(); this.success.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); this.success.write(oprot); oprot.writeFieldEnd(); } else if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getTopology_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; if (!first) sb.append(", "); sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getUserTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getUserTopology_args"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private String id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_args.class, metaDataMap); } public getUserTopology_args() { } public getUserTopology_args( String id) { this(); this.id = id; } /** * Performs a deep copy on other. */ public getUserTopology_args(getUserTopology_args other) { if (other.is_set_id()) { this.id = other.id; } } public getUserTopology_args deepCopy() { return new getUserTopology_args(this); } @Override public void clear() { this.id = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getUserTopology_args) return this.equals((getUserTopology_args)that); return false; } public boolean equals(getUserTopology_args that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); return builder.toHashCode(); } public int compareTo(getUserTopology_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getUserTopology_args typedOther = (getUserTopology_args)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getUserTopology_args("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } public static class getUserTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getUserTopology_result"); private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private StormTopology success; // required private NotAliveException e; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SUCCESS((short)0, "success"), E((short)1, "e"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; case 1: // E return E; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_result.class, metaDataMap); } public getUserTopology_result() { } public getUserTopology_result( StormTopology success, NotAliveException e) { this(); this.success = success; this.e = e; } /** * Performs a deep copy on other. */ public getUserTopology_result(getUserTopology_result other) { if (other.is_set_success()) { this.success = new StormTopology(other.success); } if (other.is_set_e()) { this.e = new NotAliveException(other.e); } } public getUserTopology_result deepCopy() { return new getUserTopology_result(this); } @Override public void clear() { this.success = null; this.e = null; } public StormTopology get_success() { return this.success; } public void set_success(StormTopology success) { this.success = success; } public void unset_success() { this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean is_set_success() { return this.success != null; } public void set_success_isSet(boolean value) { if (!value) { this.success = null; } } public NotAliveException get_e() { return this.e; } public void set_e(NotAliveException e) { this.e = e; } public void unset_e() { this.e = null; } /** Returns true if field e is set (has been assigned a value) and false otherwise */ public boolean is_set_e() { return this.e != null; } public void set_e_isSet(boolean value) { if (!value) { this.e = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unset_success(); } else { set_success((StormTopology)value); } break; case E: if (value == null) { unset_e(); } else { set_e((NotAliveException)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: return get_success(); case E: return get_e(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SUCCESS: return is_set_success(); case E: return is_set_e(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof getUserTopology_result) return this.equals((getUserTopology_result)that); return false; } public boolean equals(getUserTopology_result that) { if (that == null) return false; boolean this_present_success = true && this.is_set_success(); boolean that_present_success = true && that.is_set_success(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; if (!this.success.equals(that.success)) return false; } boolean this_present_e = true && this.is_set_e(); boolean that_present_e = true && that.is_set_e(); if (this_present_e || that_present_e) { if (!(this_present_e && that_present_e)) return false; if (!this.e.equals(that.e)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_success = true && (is_set_success()); builder.append(present_success); if (present_success) builder.append(success); boolean present_e = true && (is_set_e()); builder.append(present_e); if (present_e) builder.append(e); return builder.toHashCode(); } public int compareTo(getUserTopology_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; getUserTopology_result typedOther = (getUserTopology_result)other; lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); if (lastComparison != 0) { return lastComparison; } if (is_set_success()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); if (lastComparison != 0) { return lastComparison; } if (is_set_e()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 0: // SUCCESS if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.success = new StormTopology(); this.success.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 1: // E if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.e = new NotAliveException(); this.e.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { oprot.writeStructBegin(STRUCT_DESC); if (this.is_set_success()) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); this.success.write(oprot); oprot.writeFieldEnd(); } else if (this.is_set_e()) { oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("getUserTopology_result("); boolean first = true; sb.append("success:"); if (this.success == null) { sb.append("null"); } else { sb.append(this.success); } first = false; if (!first) sb.append(", "); sb.append("e:"); if (this.e == null) { sb.append("null"); } else { sb.append(this.e); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/NotAliveException.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class NotAliveException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("NotAliveException"); private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); private String msg; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { MSG((short)1, "msg"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // MSG return MSG; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(NotAliveException.class, metaDataMap); } public NotAliveException() { } public NotAliveException( String msg) { this(); this.msg = msg; } /** * Performs a deep copy on other. */ public NotAliveException(NotAliveException other) { if (other.is_set_msg()) { this.msg = other.msg; } } public NotAliveException deepCopy() { return new NotAliveException(this); } @Override public void clear() { this.msg = null; } public String get_msg() { return this.msg; } public void set_msg(String msg) { this.msg = msg; } public void unset_msg() { this.msg = null; } /** Returns true if field msg is set (has been assigned a value) and false otherwise */ public boolean is_set_msg() { return this.msg != null; } public void set_msg_isSet(boolean value) { if (!value) { this.msg = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case MSG: if (value == null) { unset_msg(); } else { set_msg((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case MSG: return get_msg(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case MSG: return is_set_msg(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof NotAliveException) return this.equals((NotAliveException)that); return false; } public boolean equals(NotAliveException that) { if (that == null) return false; boolean this_present_msg = true && this.is_set_msg(); boolean that_present_msg = true && that.is_set_msg(); if (this_present_msg || that_present_msg) { if (!(this_present_msg && that_present_msg)) return false; if (!this.msg.equals(that.msg)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_msg = true && (is_set_msg()); builder.append(present_msg); if (present_msg) builder.append(msg); return builder.toHashCode(); } public int compareTo(NotAliveException other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; NotAliveException typedOther = (NotAliveException)other; lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); if (lastComparison != 0) { return lastComparison; } if (is_set_msg()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // MSG if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.msg = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.msg != null) { oprot.writeFieldBegin(MSG_FIELD_DESC); oprot.writeString(this.msg); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("NotAliveException("); boolean first = true; sb.append("msg:"); if (this.msg == null) { sb.append("null"); } else { sb.append(this.msg); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_msg()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/NullStruct.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class NullStruct implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("NullStruct"); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ; private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(NullStruct.class, metaDataMap); } public NullStruct() { } /** * Performs a deep copy on other. */ public NullStruct(NullStruct other) { } public NullStruct deepCopy() { return new NullStruct(this); } @Override public void clear() { } public void setFieldValue(_Fields field, Object value) { switch (field) { } } public Object getFieldValue(_Fields field) { switch (field) { } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof NullStruct) return this.equals((NullStruct)that); return false; } public boolean equals(NullStruct that) { if (that == null) return false; return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); return builder.toHashCode(); } public int compareTo(NullStruct other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; NullStruct typedOther = (NullStruct)other; return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("NullStruct("); boolean first = true; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class RebalanceOptions implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("RebalanceOptions"); private static final org.apache.thrift7.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("wait_secs", org.apache.thrift7.protocol.TType.I32, (short)1); private static final org.apache.thrift7.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_workers", org.apache.thrift7.protocol.TType.I32, (short)2); private static final org.apache.thrift7.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_executors", org.apache.thrift7.protocol.TType.MAP, (short)3); private int wait_secs; // required private int num_workers; // required private Map num_executors; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { WAIT_SECS((short)1, "wait_secs"), NUM_WORKERS((short)2, "num_workers"), NUM_EXECUTORS((short)3, "num_executors"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // WAIT_SECS return WAIT_SECS; case 2: // NUM_WORKERS return NUM_WORKERS; case 3: // NUM_EXECUTORS return NUM_EXECUTORS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __WAIT_SECS_ISSET_ID = 0; private static final int __NUM_WORKERS_ISSET_ID = 1; private BitSet __isset_bit_vector = new BitSet(2); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift7.meta_data.FieldMetaData("wait_secs", org.apache.thrift7.TFieldRequirementType.OPTIONAL, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_workers", org.apache.thrift7.TFieldRequirementType.OPTIONAL, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift7.meta_data.FieldMetaData("num_executors", org.apache.thrift7.TFieldRequirementType.OPTIONAL, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32)))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(RebalanceOptions.class, metaDataMap); } public RebalanceOptions() { } /** * Performs a deep copy on other. */ public RebalanceOptions(RebalanceOptions other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); this.wait_secs = other.wait_secs; this.num_workers = other.num_workers; if (other.is_set_num_executors()) { Map __this__num_executors = new HashMap(); for (Map.Entry other_element : other.num_executors.entrySet()) { String other_element_key = other_element.getKey(); Integer other_element_value = other_element.getValue(); String __this__num_executors_copy_key = other_element_key; Integer __this__num_executors_copy_value = other_element_value; __this__num_executors.put(__this__num_executors_copy_key, __this__num_executors_copy_value); } this.num_executors = __this__num_executors; } } public RebalanceOptions deepCopy() { return new RebalanceOptions(this); } @Override public void clear() { set_wait_secs_isSet(false); this.wait_secs = 0; set_num_workers_isSet(false); this.num_workers = 0; this.num_executors = null; } public int get_wait_secs() { return this.wait_secs; } public void set_wait_secs(int wait_secs) { this.wait_secs = wait_secs; set_wait_secs_isSet(true); } public void unset_wait_secs() { __isset_bit_vector.clear(__WAIT_SECS_ISSET_ID); } /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_wait_secs() { return __isset_bit_vector.get(__WAIT_SECS_ISSET_ID); } public void set_wait_secs_isSet(boolean value) { __isset_bit_vector.set(__WAIT_SECS_ISSET_ID, value); } public int get_num_workers() { return this.num_workers; } public void set_num_workers(int num_workers) { this.num_workers = num_workers; set_num_workers_isSet(true); } public void unset_num_workers() { __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID); } /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ public boolean is_set_num_workers() { return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID); } public void set_num_workers_isSet(boolean value) { __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value); } public int get_num_executors_size() { return (this.num_executors == null) ? 0 : this.num_executors.size(); } public void put_to_num_executors(String key, int val) { if (this.num_executors == null) { this.num_executors = new HashMap(); } this.num_executors.put(key, val); } public Map get_num_executors() { return this.num_executors; } public void set_num_executors(Map num_executors) { this.num_executors = num_executors; } public void unset_num_executors() { this.num_executors = null; } /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */ public boolean is_set_num_executors() { return this.num_executors != null; } public void set_num_executors_isSet(boolean value) { if (!value) { this.num_executors = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case WAIT_SECS: if (value == null) { unset_wait_secs(); } else { set_wait_secs((Integer)value); } break; case NUM_WORKERS: if (value == null) { unset_num_workers(); } else { set_num_workers((Integer)value); } break; case NUM_EXECUTORS: if (value == null) { unset_num_executors(); } else { set_num_executors((Map)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case WAIT_SECS: return Integer.valueOf(get_wait_secs()); case NUM_WORKERS: return Integer.valueOf(get_num_workers()); case NUM_EXECUTORS: return get_num_executors(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case WAIT_SECS: return is_set_wait_secs(); case NUM_WORKERS: return is_set_num_workers(); case NUM_EXECUTORS: return is_set_num_executors(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof RebalanceOptions) return this.equals((RebalanceOptions)that); return false; } public boolean equals(RebalanceOptions that) { if (that == null) return false; boolean this_present_wait_secs = true && this.is_set_wait_secs(); boolean that_present_wait_secs = true && that.is_set_wait_secs(); if (this_present_wait_secs || that_present_wait_secs) { if (!(this_present_wait_secs && that_present_wait_secs)) return false; if (this.wait_secs != that.wait_secs) return false; } boolean this_present_num_workers = true && this.is_set_num_workers(); boolean that_present_num_workers = true && that.is_set_num_workers(); if (this_present_num_workers || that_present_num_workers) { if (!(this_present_num_workers && that_present_num_workers)) return false; if (this.num_workers != that.num_workers) return false; } boolean this_present_num_executors = true && this.is_set_num_executors(); boolean that_present_num_executors = true && that.is_set_num_executors(); if (this_present_num_executors || that_present_num_executors) { if (!(this_present_num_executors && that_present_num_executors)) return false; if (!this.num_executors.equals(that.num_executors)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_wait_secs = true && (is_set_wait_secs()); builder.append(present_wait_secs); if (present_wait_secs) builder.append(wait_secs); boolean present_num_workers = true && (is_set_num_workers()); builder.append(present_num_workers); if (present_num_workers) builder.append(num_workers); boolean present_num_executors = true && (is_set_num_executors()); builder.append(present_num_executors); if (present_num_executors) builder.append(num_executors); return builder.toHashCode(); } public int compareTo(RebalanceOptions other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; RebalanceOptions typedOther = (RebalanceOptions)other; lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(typedOther.is_set_wait_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_wait_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.wait_secs, typedOther.wait_secs); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers()); if (lastComparison != 0) { return lastComparison; } if (is_set_num_workers()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(typedOther.is_set_num_executors()); if (lastComparison != 0) { return lastComparison; } if (is_set_num_executors()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_executors, typedOther.num_executors); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // WAIT_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.wait_secs = iprot.readI32(); set_wait_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // NUM_WORKERS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.num_workers = iprot.readI32(); set_num_workers_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // NUM_EXECUTORS if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map158 = iprot.readMapBegin(); this.num_executors = new HashMap(2*_map158.size); for (int _i159 = 0; _i159 < _map158.size; ++_i159) { String _key160; // required int _val161; // required _key160 = iprot.readString(); _val161 = iprot.readI32(); this.num_executors.put(_key160, _val161); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (is_set_wait_secs()) { oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC); oprot.writeI32(this.wait_secs); oprot.writeFieldEnd(); } if (is_set_num_workers()) { oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); oprot.writeI32(this.num_workers); oprot.writeFieldEnd(); } if (this.num_executors != null) { if (is_set_num_executors()) { oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I32, this.num_executors.size())); for (Map.Entry _iter162 : this.num_executors.entrySet()) { oprot.writeString(_iter162.getKey()); oprot.writeI32(_iter162.getValue()); } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("RebalanceOptions("); boolean first = true; if (is_set_wait_secs()) { sb.append("wait_secs:"); sb.append(this.wait_secs); first = false; } if (is_set_num_workers()) { if (!first) sb.append(", "); sb.append("num_workers:"); sb.append(this.num_workers); first = false; } if (is_set_num_executors()) { if (!first) sb.append(", "); sb.append("num_executors:"); if (this.num_executors == null) { sb.append("null"); } else { sb.append(this.num_executors); } first = false; } sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/ShellComponent.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ShellComponent implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ShellComponent"); private static final org.apache.thrift7.protocol.TField EXECUTION_COMMAND_FIELD_DESC = new org.apache.thrift7.protocol.TField("execution_command", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField SCRIPT_FIELD_DESC = new org.apache.thrift7.protocol.TField("script", org.apache.thrift7.protocol.TType.STRING, (short)2); private String execution_command; // required private String script; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { EXECUTION_COMMAND((short)1, "execution_command"), SCRIPT((short)2, "script"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // EXECUTION_COMMAND return EXECUTION_COMMAND; case 2: // SCRIPT return SCRIPT; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.EXECUTION_COMMAND, new org.apache.thrift7.meta_data.FieldMetaData("execution_command", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.SCRIPT, new org.apache.thrift7.meta_data.FieldMetaData("script", org.apache.thrift7.TFieldRequirementType.DEFAULT, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ShellComponent.class, metaDataMap); } public ShellComponent() { } public ShellComponent( String execution_command, String script) { this(); this.execution_command = execution_command; this.script = script; } /** * Performs a deep copy on other. */ public ShellComponent(ShellComponent other) { if (other.is_set_execution_command()) { this.execution_command = other.execution_command; } if (other.is_set_script()) { this.script = other.script; } } public ShellComponent deepCopy() { return new ShellComponent(this); } @Override public void clear() { this.execution_command = null; this.script = null; } public String get_execution_command() { return this.execution_command; } public void set_execution_command(String execution_command) { this.execution_command = execution_command; } public void unset_execution_command() { this.execution_command = null; } /** Returns true if field execution_command is set (has been assigned a value) and false otherwise */ public boolean is_set_execution_command() { return this.execution_command != null; } public void set_execution_command_isSet(boolean value) { if (!value) { this.execution_command = null; } } public String get_script() { return this.script; } public void set_script(String script) { this.script = script; } public void unset_script() { this.script = null; } /** Returns true if field script is set (has been assigned a value) and false otherwise */ public boolean is_set_script() { return this.script != null; } public void set_script_isSet(boolean value) { if (!value) { this.script = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case EXECUTION_COMMAND: if (value == null) { unset_execution_command(); } else { set_execution_command((String)value); } break; case SCRIPT: if (value == null) { unset_script(); } else { set_script((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case EXECUTION_COMMAND: return get_execution_command(); case SCRIPT: return get_script(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case EXECUTION_COMMAND: return is_set_execution_command(); case SCRIPT: return is_set_script(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof ShellComponent) return this.equals((ShellComponent)that); return false; } public boolean equals(ShellComponent that) { if (that == null) return false; boolean this_present_execution_command = true && this.is_set_execution_command(); boolean that_present_execution_command = true && that.is_set_execution_command(); if (this_present_execution_command || that_present_execution_command) { if (!(this_present_execution_command && that_present_execution_command)) return false; if (!this.execution_command.equals(that.execution_command)) return false; } boolean this_present_script = true && this.is_set_script(); boolean that_present_script = true && that.is_set_script(); if (this_present_script || that_present_script) { if (!(this_present_script && that_present_script)) return false; if (!this.script.equals(that.script)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_execution_command = true && (is_set_execution_command()); builder.append(present_execution_command); if (present_execution_command) builder.append(execution_command); boolean present_script = true && (is_set_script()); builder.append(present_script); if (present_script) builder.append(script); return builder.toHashCode(); } public int compareTo(ShellComponent other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; ShellComponent typedOther = (ShellComponent)other; lastComparison = Boolean.valueOf(is_set_execution_command()).compareTo(typedOther.is_set_execution_command()); if (lastComparison != 0) { return lastComparison; } if (is_set_execution_command()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.execution_command, typedOther.execution_command); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_script()).compareTo(typedOther.is_set_script()); if (lastComparison != 0) { return lastComparison; } if (is_set_script()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.script, typedOther.script); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // EXECUTION_COMMAND if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.execution_command = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // SCRIPT if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.script = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.execution_command != null) { oprot.writeFieldBegin(EXECUTION_COMMAND_FIELD_DESC); oprot.writeString(this.execution_command); oprot.writeFieldEnd(); } if (this.script != null) { oprot.writeFieldBegin(SCRIPT_FIELD_DESC); oprot.writeString(this.script); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("ShellComponent("); boolean first = true; sb.append("execution_command:"); if (this.execution_command == null) { sb.append("null"); } else { sb.append(this.execution_command); } first = false; if (!first) sb.append(", "); sb.append("script:"); if (this.script == null) { sb.append("null"); } else { sb.append(this.script); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SpoutSpec implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SpoutSpec"); private static final org.apache.thrift7.protocol.TField SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("spout_object", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift7.protocol.TField("common", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private ComponentObject spout_object; // required private ComponentCommon common; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SPOUT_OBJECT((short)1, "spout_object"), COMMON((short)2, "common"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // SPOUT_OBJECT return SPOUT_OBJECT; case 2: // COMMON return COMMON; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SPOUT_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("spout_object", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentObject.class))); tmpMap.put(_Fields.COMMON, new org.apache.thrift7.meta_data.FieldMetaData("common", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentCommon.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SpoutSpec.class, metaDataMap); } public SpoutSpec() { } public SpoutSpec( ComponentObject spout_object, ComponentCommon common) { this(); this.spout_object = spout_object; this.common = common; } /** * Performs a deep copy on other. */ public SpoutSpec(SpoutSpec other) { if (other.is_set_spout_object()) { this.spout_object = new ComponentObject(other.spout_object); } if (other.is_set_common()) { this.common = new ComponentCommon(other.common); } } public SpoutSpec deepCopy() { return new SpoutSpec(this); } @Override public void clear() { this.spout_object = null; this.common = null; } public ComponentObject get_spout_object() { return this.spout_object; } public void set_spout_object(ComponentObject spout_object) { this.spout_object = spout_object; } public void unset_spout_object() { this.spout_object = null; } /** Returns true if field spout_object is set (has been assigned a value) and false otherwise */ public boolean is_set_spout_object() { return this.spout_object != null; } public void set_spout_object_isSet(boolean value) { if (!value) { this.spout_object = null; } } public ComponentCommon get_common() { return this.common; } public void set_common(ComponentCommon common) { this.common = common; } public void unset_common() { this.common = null; } /** Returns true if field common is set (has been assigned a value) and false otherwise */ public boolean is_set_common() { return this.common != null; } public void set_common_isSet(boolean value) { if (!value) { this.common = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SPOUT_OBJECT: if (value == null) { unset_spout_object(); } else { set_spout_object((ComponentObject)value); } break; case COMMON: if (value == null) { unset_common(); } else { set_common((ComponentCommon)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SPOUT_OBJECT: return get_spout_object(); case COMMON: return get_common(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SPOUT_OBJECT: return is_set_spout_object(); case COMMON: return is_set_common(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof SpoutSpec) return this.equals((SpoutSpec)that); return false; } public boolean equals(SpoutSpec that) { if (that == null) return false; boolean this_present_spout_object = true && this.is_set_spout_object(); boolean that_present_spout_object = true && that.is_set_spout_object(); if (this_present_spout_object || that_present_spout_object) { if (!(this_present_spout_object && that_present_spout_object)) return false; if (!this.spout_object.equals(that.spout_object)) return false; } boolean this_present_common = true && this.is_set_common(); boolean that_present_common = true && that.is_set_common(); if (this_present_common || that_present_common) { if (!(this_present_common && that_present_common)) return false; if (!this.common.equals(that.common)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_spout_object = true && (is_set_spout_object()); builder.append(present_spout_object); if (present_spout_object) builder.append(spout_object); boolean present_common = true && (is_set_common()); builder.append(present_common); if (present_common) builder.append(common); return builder.toHashCode(); } public int compareTo(SpoutSpec other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; SpoutSpec typedOther = (SpoutSpec)other; lastComparison = Boolean.valueOf(is_set_spout_object()).compareTo(typedOther.is_set_spout_object()); if (lastComparison != 0) { return lastComparison; } if (is_set_spout_object()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.spout_object, typedOther.spout_object); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common()); if (lastComparison != 0) { return lastComparison; } if (is_set_common()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.common, typedOther.common); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // SPOUT_OBJECT if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.spout_object = new ComponentObject(); this.spout_object.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // COMMON if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.common = new ComponentCommon(); this.common.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.spout_object != null) { oprot.writeFieldBegin(SPOUT_OBJECT_FIELD_DESC); this.spout_object.write(oprot); oprot.writeFieldEnd(); } if (this.common != null) { oprot.writeFieldBegin(COMMON_FIELD_DESC); this.common.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("SpoutSpec("); boolean first = true; sb.append("spout_object:"); if (this.spout_object == null) { sb.append("null"); } else { sb.append(this.spout_object); } first = false; if (!first) sb.append(", "); sb.append("common:"); if (this.common == null) { sb.append("null"); } else { sb.append(this.common); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_spout_object()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'spout_object' is unset! Struct:" + toString()); } if (!is_set_common()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/SpoutStats.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SpoutStats implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SpoutStats"); private static final org.apache.thrift7.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift7.protocol.TField("acked", org.apache.thrift7.protocol.TType.MAP, (short)1); private static final org.apache.thrift7.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift7.protocol.TField("failed", org.apache.thrift7.protocol.TType.MAP, (short)2); private static final org.apache.thrift7.protocol.TField COMPLETE_MS_AVG_FIELD_DESC = new org.apache.thrift7.protocol.TField("complete_ms_avg", org.apache.thrift7.protocol.TType.MAP, (short)3); private Map> acked; // required private Map> failed; // required private Map> complete_ms_avg; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ACKED((short)1, "acked"), FAILED((short)2, "failed"), COMPLETE_MS_AVG((short)3, "complete_ms_avg"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ACKED return ACKED; case 2: // FAILED return FAILED; case 3: // COMPLETE_MS_AVG return COMPLETE_MS_AVG; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ACKED, new org.apache.thrift7.meta_data.FieldMetaData("acked", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); tmpMap.put(_Fields.FAILED, new org.apache.thrift7.meta_data.FieldMetaData("failed", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); tmpMap.put(_Fields.COMPLETE_MS_AVG, new org.apache.thrift7.meta_data.FieldMetaData("complete_ms_avg", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SpoutStats.class, metaDataMap); } public SpoutStats() { } public SpoutStats( Map> acked, Map> failed, Map> complete_ms_avg) { this(); this.acked = acked; this.failed = failed; this.complete_ms_avg = complete_ms_avg; } /** * Performs a deep copy on other. */ public SpoutStats(SpoutStats other) { if (other.is_set_acked()) { Map> __this__acked = new HashMap>(); for (Map.Entry> other_element : other.acked.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__acked_copy_key = other_element_key; Map __this__acked_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { String other_element_value_element_key = other_element_value_element.getKey(); Long other_element_value_element_value = other_element_value_element.getValue(); String __this__acked_copy_value_copy_key = other_element_value_element_key; Long __this__acked_copy_value_copy_value = other_element_value_element_value; __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value); } __this__acked.put(__this__acked_copy_key, __this__acked_copy_value); } this.acked = __this__acked; } if (other.is_set_failed()) { Map> __this__failed = new HashMap>(); for (Map.Entry> other_element : other.failed.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__failed_copy_key = other_element_key; Map __this__failed_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { String other_element_value_element_key = other_element_value_element.getKey(); Long other_element_value_element_value = other_element_value_element.getValue(); String __this__failed_copy_value_copy_key = other_element_value_element_key; Long __this__failed_copy_value_copy_value = other_element_value_element_value; __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value); } __this__failed.put(__this__failed_copy_key, __this__failed_copy_value); } this.failed = __this__failed; } if (other.is_set_complete_ms_avg()) { Map> __this__complete_ms_avg = new HashMap>(); for (Map.Entry> other_element : other.complete_ms_avg.entrySet()) { String other_element_key = other_element.getKey(); Map other_element_value = other_element.getValue(); String __this__complete_ms_avg_copy_key = other_element_key; Map __this__complete_ms_avg_copy_value = new HashMap(); for (Map.Entry other_element_value_element : other_element_value.entrySet()) { String other_element_value_element_key = other_element_value_element.getKey(); Double other_element_value_element_value = other_element_value_element.getValue(); String __this__complete_ms_avg_copy_value_copy_key = other_element_value_element_key; Double __this__complete_ms_avg_copy_value_copy_value = other_element_value_element_value; __this__complete_ms_avg_copy_value.put(__this__complete_ms_avg_copy_value_copy_key, __this__complete_ms_avg_copy_value_copy_value); } __this__complete_ms_avg.put(__this__complete_ms_avg_copy_key, __this__complete_ms_avg_copy_value); } this.complete_ms_avg = __this__complete_ms_avg; } } public SpoutStats deepCopy() { return new SpoutStats(this); } @Override public void clear() { this.acked = null; this.failed = null; this.complete_ms_avg = null; } public int get_acked_size() { return (this.acked == null) ? 0 : this.acked.size(); } public void put_to_acked(String key, Map val) { if (this.acked == null) { this.acked = new HashMap>(); } this.acked.put(key, val); } public Map> get_acked() { return this.acked; } public void set_acked(Map> acked) { this.acked = acked; } public void unset_acked() { this.acked = null; } /** Returns true if field acked is set (has been assigned a value) and false otherwise */ public boolean is_set_acked() { return this.acked != null; } public void set_acked_isSet(boolean value) { if (!value) { this.acked = null; } } public int get_failed_size() { return (this.failed == null) ? 0 : this.failed.size(); } public void put_to_failed(String key, Map val) { if (this.failed == null) { this.failed = new HashMap>(); } this.failed.put(key, val); } public Map> get_failed() { return this.failed; } public void set_failed(Map> failed) { this.failed = failed; } public void unset_failed() { this.failed = null; } /** Returns true if field failed is set (has been assigned a value) and false otherwise */ public boolean is_set_failed() { return this.failed != null; } public void set_failed_isSet(boolean value) { if (!value) { this.failed = null; } } public int get_complete_ms_avg_size() { return (this.complete_ms_avg == null) ? 0 : this.complete_ms_avg.size(); } public void put_to_complete_ms_avg(String key, Map val) { if (this.complete_ms_avg == null) { this.complete_ms_avg = new HashMap>(); } this.complete_ms_avg.put(key, val); } public Map> get_complete_ms_avg() { return this.complete_ms_avg; } public void set_complete_ms_avg(Map> complete_ms_avg) { this.complete_ms_avg = complete_ms_avg; } public void unset_complete_ms_avg() { this.complete_ms_avg = null; } /** Returns true if field complete_ms_avg is set (has been assigned a value) and false otherwise */ public boolean is_set_complete_ms_avg() { return this.complete_ms_avg != null; } public void set_complete_ms_avg_isSet(boolean value) { if (!value) { this.complete_ms_avg = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ACKED: if (value == null) { unset_acked(); } else { set_acked((Map>)value); } break; case FAILED: if (value == null) { unset_failed(); } else { set_failed((Map>)value); } break; case COMPLETE_MS_AVG: if (value == null) { unset_complete_ms_avg(); } else { set_complete_ms_avg((Map>)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ACKED: return get_acked(); case FAILED: return get_failed(); case COMPLETE_MS_AVG: return get_complete_ms_avg(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ACKED: return is_set_acked(); case FAILED: return is_set_failed(); case COMPLETE_MS_AVG: return is_set_complete_ms_avg(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof SpoutStats) return this.equals((SpoutStats)that); return false; } public boolean equals(SpoutStats that) { if (that == null) return false; boolean this_present_acked = true && this.is_set_acked(); boolean that_present_acked = true && that.is_set_acked(); if (this_present_acked || that_present_acked) { if (!(this_present_acked && that_present_acked)) return false; if (!this.acked.equals(that.acked)) return false; } boolean this_present_failed = true && this.is_set_failed(); boolean that_present_failed = true && that.is_set_failed(); if (this_present_failed || that_present_failed) { if (!(this_present_failed && that_present_failed)) return false; if (!this.failed.equals(that.failed)) return false; } boolean this_present_complete_ms_avg = true && this.is_set_complete_ms_avg(); boolean that_present_complete_ms_avg = true && that.is_set_complete_ms_avg(); if (this_present_complete_ms_avg || that_present_complete_ms_avg) { if (!(this_present_complete_ms_avg && that_present_complete_ms_avg)) return false; if (!this.complete_ms_avg.equals(that.complete_ms_avg)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_acked = true && (is_set_acked()); builder.append(present_acked); if (present_acked) builder.append(acked); boolean present_failed = true && (is_set_failed()); builder.append(present_failed); if (present_failed) builder.append(failed); boolean present_complete_ms_avg = true && (is_set_complete_ms_avg()); builder.append(present_complete_ms_avg); if (present_complete_ms_avg) builder.append(complete_ms_avg); return builder.toHashCode(); } public int compareTo(SpoutStats other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; SpoutStats typedOther = (SpoutStats)other; lastComparison = Boolean.valueOf(is_set_acked()).compareTo(typedOther.is_set_acked()); if (lastComparison != 0) { return lastComparison; } if (is_set_acked()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.acked, typedOther.acked); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_failed()).compareTo(typedOther.is_set_failed()); if (lastComparison != 0) { return lastComparison; } if (is_set_failed()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.failed, typedOther.failed); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_complete_ms_avg()).compareTo(typedOther.is_set_complete_ms_avg()); if (lastComparison != 0) { return lastComparison; } if (is_set_complete_ms_avg()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.complete_ms_avg, typedOther.complete_ms_avg); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ACKED if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map95 = iprot.readMapBegin(); this.acked = new HashMap>(2*_map95.size); for (int _i96 = 0; _i96 < _map95.size; ++_i96) { String _key97; // required Map _val98; // required _key97 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map99 = iprot.readMapBegin(); _val98 = new HashMap(2*_map99.size); for (int _i100 = 0; _i100 < _map99.size; ++_i100) { String _key101; // required long _val102; // required _key101 = iprot.readString(); _val102 = iprot.readI64(); _val98.put(_key101, _val102); } iprot.readMapEnd(); } this.acked.put(_key97, _val98); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // FAILED if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map103 = iprot.readMapBegin(); this.failed = new HashMap>(2*_map103.size); for (int _i104 = 0; _i104 < _map103.size; ++_i104) { String _key105; // required Map _val106; // required _key105 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map107 = iprot.readMapBegin(); _val106 = new HashMap(2*_map107.size); for (int _i108 = 0; _i108 < _map107.size; ++_i108) { String _key109; // required long _val110; // required _key109 = iprot.readString(); _val110 = iprot.readI64(); _val106.put(_key109, _val110); } iprot.readMapEnd(); } this.failed.put(_key105, _val106); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // COMPLETE_MS_AVG if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map111 = iprot.readMapBegin(); this.complete_ms_avg = new HashMap>(2*_map111.size); for (int _i112 = 0; _i112 < _map111.size; ++_i112) { String _key113; // required Map _val114; // required _key113 = iprot.readString(); { org.apache.thrift7.protocol.TMap _map115 = iprot.readMapBegin(); _val114 = new HashMap(2*_map115.size); for (int _i116 = 0; _i116 < _map115.size; ++_i116) { String _key117; // required double _val118; // required _key117 = iprot.readString(); _val118 = iprot.readDouble(); _val114.put(_key117, _val118); } iprot.readMapEnd(); } this.complete_ms_avg.put(_key113, _val114); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.acked != null) { oprot.writeFieldBegin(ACKED_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.acked.size())); for (Map.Entry> _iter119 : this.acked.entrySet()) { oprot.writeString(_iter119.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I64, _iter119.getValue().size())); for (Map.Entry _iter120 : _iter119.getValue().entrySet()) { oprot.writeString(_iter120.getKey()); oprot.writeI64(_iter120.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.failed != null) { oprot.writeFieldBegin(FAILED_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.failed.size())); for (Map.Entry> _iter121 : this.failed.entrySet()) { oprot.writeString(_iter121.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I64, _iter121.getValue().size())); for (Map.Entry _iter122 : _iter121.getValue().entrySet()) { oprot.writeString(_iter122.getKey()); oprot.writeI64(_iter122.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.complete_ms_avg != null) { oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.complete_ms_avg.size())); for (Map.Entry> _iter123 : this.complete_ms_avg.entrySet()) { oprot.writeString(_iter123.getKey()); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, _iter123.getValue().size())); for (Map.Entry _iter124 : _iter123.getValue().entrySet()) { oprot.writeString(_iter124.getKey()); oprot.writeDouble(_iter124.getValue()); } oprot.writeMapEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("SpoutStats("); boolean first = true; sb.append("acked:"); if (this.acked == null) { sb.append("null"); } else { sb.append(this.acked); } first = false; if (!first) sb.append(", "); sb.append("failed:"); if (this.failed == null) { sb.append("null"); } else { sb.append(this.failed); } first = false; if (!first) sb.append(", "); sb.append("complete_ms_avg:"); if (this.complete_ms_avg == null) { sb.append("null"); } else { sb.append(this.complete_ms_avg); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_acked()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'acked' is unset! Struct:" + toString()); } if (!is_set_failed()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'failed' is unset! Struct:" + toString()); } if (!is_set_complete_ms_avg()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'complete_ms_avg' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class StateSpoutSpec implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("StateSpoutSpec"); private static final org.apache.thrift7.protocol.TField STATE_SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("state_spout_object", org.apache.thrift7.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift7.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift7.protocol.TField("common", org.apache.thrift7.protocol.TType.STRUCT, (short)2); private ComponentObject state_spout_object; // required private ComponentCommon common; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { STATE_SPOUT_OBJECT((short)1, "state_spout_object"), COMMON((short)2, "common"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // STATE_SPOUT_OBJECT return STATE_SPOUT_OBJECT; case 2: // COMMON return COMMON; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.STATE_SPOUT_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("state_spout_object", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentObject.class))); tmpMap.put(_Fields.COMMON, new org.apache.thrift7.meta_data.FieldMetaData("common", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentCommon.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(StateSpoutSpec.class, metaDataMap); } public StateSpoutSpec() { } public StateSpoutSpec( ComponentObject state_spout_object, ComponentCommon common) { this(); this.state_spout_object = state_spout_object; this.common = common; } /** * Performs a deep copy on other. */ public StateSpoutSpec(StateSpoutSpec other) { if (other.is_set_state_spout_object()) { this.state_spout_object = new ComponentObject(other.state_spout_object); } if (other.is_set_common()) { this.common = new ComponentCommon(other.common); } } public StateSpoutSpec deepCopy() { return new StateSpoutSpec(this); } @Override public void clear() { this.state_spout_object = null; this.common = null; } public ComponentObject get_state_spout_object() { return this.state_spout_object; } public void set_state_spout_object(ComponentObject state_spout_object) { this.state_spout_object = state_spout_object; } public void unset_state_spout_object() { this.state_spout_object = null; } /** Returns true if field state_spout_object is set (has been assigned a value) and false otherwise */ public boolean is_set_state_spout_object() { return this.state_spout_object != null; } public void set_state_spout_object_isSet(boolean value) { if (!value) { this.state_spout_object = null; } } public ComponentCommon get_common() { return this.common; } public void set_common(ComponentCommon common) { this.common = common; } public void unset_common() { this.common = null; } /** Returns true if field common is set (has been assigned a value) and false otherwise */ public boolean is_set_common() { return this.common != null; } public void set_common_isSet(boolean value) { if (!value) { this.common = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case STATE_SPOUT_OBJECT: if (value == null) { unset_state_spout_object(); } else { set_state_spout_object((ComponentObject)value); } break; case COMMON: if (value == null) { unset_common(); } else { set_common((ComponentCommon)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case STATE_SPOUT_OBJECT: return get_state_spout_object(); case COMMON: return get_common(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case STATE_SPOUT_OBJECT: return is_set_state_spout_object(); case COMMON: return is_set_common(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof StateSpoutSpec) return this.equals((StateSpoutSpec)that); return false; } public boolean equals(StateSpoutSpec that) { if (that == null) return false; boolean this_present_state_spout_object = true && this.is_set_state_spout_object(); boolean that_present_state_spout_object = true && that.is_set_state_spout_object(); if (this_present_state_spout_object || that_present_state_spout_object) { if (!(this_present_state_spout_object && that_present_state_spout_object)) return false; if (!this.state_spout_object.equals(that.state_spout_object)) return false; } boolean this_present_common = true && this.is_set_common(); boolean that_present_common = true && that.is_set_common(); if (this_present_common || that_present_common) { if (!(this_present_common && that_present_common)) return false; if (!this.common.equals(that.common)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_state_spout_object = true && (is_set_state_spout_object()); builder.append(present_state_spout_object); if (present_state_spout_object) builder.append(state_spout_object); boolean present_common = true && (is_set_common()); builder.append(present_common); if (present_common) builder.append(common); return builder.toHashCode(); } public int compareTo(StateSpoutSpec other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; StateSpoutSpec typedOther = (StateSpoutSpec)other; lastComparison = Boolean.valueOf(is_set_state_spout_object()).compareTo(typedOther.is_set_state_spout_object()); if (lastComparison != 0) { return lastComparison; } if (is_set_state_spout_object()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.state_spout_object, typedOther.state_spout_object); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common()); if (lastComparison != 0) { return lastComparison; } if (is_set_common()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.common, typedOther.common); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // STATE_SPOUT_OBJECT if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.state_spout_object = new ComponentObject(); this.state_spout_object.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // COMMON if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { this.common = new ComponentCommon(); this.common.read(iprot); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.state_spout_object != null) { oprot.writeFieldBegin(STATE_SPOUT_OBJECT_FIELD_DESC); this.state_spout_object.write(oprot); oprot.writeFieldEnd(); } if (this.common != null) { oprot.writeFieldBegin(COMMON_FIELD_DESC); this.common.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("StateSpoutSpec("); boolean first = true; sb.append("state_spout_object:"); if (this.state_spout_object == null) { sb.append("null"); } else { sb.append(this.state_spout_object); } first = false; if (!first) sb.append(", "); sb.append("common:"); if (this.common == null) { sb.append("null"); } else { sb.append(this.common); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_state_spout_object()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'state_spout_object' is unset! Struct:" + toString()); } if (!is_set_common()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/StormTopology.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class StormTopology implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("StormTopology"); private static final org.apache.thrift7.protocol.TField SPOUTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("spouts", org.apache.thrift7.protocol.TType.MAP, (short)1); private static final org.apache.thrift7.protocol.TField BOLTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("bolts", org.apache.thrift7.protocol.TType.MAP, (short)2); private static final org.apache.thrift7.protocol.TField STATE_SPOUTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("state_spouts", org.apache.thrift7.protocol.TType.MAP, (short)3); private Map spouts; // required private Map bolts; // required private Map state_spouts; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { SPOUTS((short)1, "spouts"), BOLTS((short)2, "bolts"), STATE_SPOUTS((short)3, "state_spouts"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // SPOUTS return SPOUTS; case 2: // BOLTS return BOLTS; case 3: // STATE_SPOUTS return STATE_SPOUTS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SPOUTS, new org.apache.thrift7.meta_data.FieldMetaData("spouts", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SpoutSpec.class)))); tmpMap.put(_Fields.BOLTS, new org.apache.thrift7.meta_data.FieldMetaData("bolts", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, Bolt.class)))); tmpMap.put(_Fields.STATE_SPOUTS, new org.apache.thrift7.meta_data.FieldMetaData("state_spouts", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StateSpoutSpec.class)))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(StormTopology.class, metaDataMap); } public StormTopology() { } public StormTopology( Map spouts, Map bolts, Map state_spouts) { this(); this.spouts = spouts; this.bolts = bolts; this.state_spouts = state_spouts; } /** * Performs a deep copy on other. */ public StormTopology(StormTopology other) { if (other.is_set_spouts()) { Map __this__spouts = new HashMap(); for (Map.Entry other_element : other.spouts.entrySet()) { String other_element_key = other_element.getKey(); SpoutSpec other_element_value = other_element.getValue(); String __this__spouts_copy_key = other_element_key; SpoutSpec __this__spouts_copy_value = new SpoutSpec(other_element_value); __this__spouts.put(__this__spouts_copy_key, __this__spouts_copy_value); } this.spouts = __this__spouts; } if (other.is_set_bolts()) { Map __this__bolts = new HashMap(); for (Map.Entry other_element : other.bolts.entrySet()) { String other_element_key = other_element.getKey(); Bolt other_element_value = other_element.getValue(); String __this__bolts_copy_key = other_element_key; Bolt __this__bolts_copy_value = new Bolt(other_element_value); __this__bolts.put(__this__bolts_copy_key, __this__bolts_copy_value); } this.bolts = __this__bolts; } if (other.is_set_state_spouts()) { Map __this__state_spouts = new HashMap(); for (Map.Entry other_element : other.state_spouts.entrySet()) { String other_element_key = other_element.getKey(); StateSpoutSpec other_element_value = other_element.getValue(); String __this__state_spouts_copy_key = other_element_key; StateSpoutSpec __this__state_spouts_copy_value = new StateSpoutSpec(other_element_value); __this__state_spouts.put(__this__state_spouts_copy_key, __this__state_spouts_copy_value); } this.state_spouts = __this__state_spouts; } } public StormTopology deepCopy() { return new StormTopology(this); } @Override public void clear() { this.spouts = null; this.bolts = null; this.state_spouts = null; } public int get_spouts_size() { return (this.spouts == null) ? 0 : this.spouts.size(); } public void put_to_spouts(String key, SpoutSpec val) { if (this.spouts == null) { this.spouts = new HashMap(); } this.spouts.put(key, val); } public Map get_spouts() { return this.spouts; } public void set_spouts(Map spouts) { this.spouts = spouts; } public void unset_spouts() { this.spouts = null; } /** Returns true if field spouts is set (has been assigned a value) and false otherwise */ public boolean is_set_spouts() { return this.spouts != null; } public void set_spouts_isSet(boolean value) { if (!value) { this.spouts = null; } } public int get_bolts_size() { return (this.bolts == null) ? 0 : this.bolts.size(); } public void put_to_bolts(String key, Bolt val) { if (this.bolts == null) { this.bolts = new HashMap(); } this.bolts.put(key, val); } public Map get_bolts() { return this.bolts; } public void set_bolts(Map bolts) { this.bolts = bolts; } public void unset_bolts() { this.bolts = null; } /** Returns true if field bolts is set (has been assigned a value) and false otherwise */ public boolean is_set_bolts() { return this.bolts != null; } public void set_bolts_isSet(boolean value) { if (!value) { this.bolts = null; } } public int get_state_spouts_size() { return (this.state_spouts == null) ? 0 : this.state_spouts.size(); } public void put_to_state_spouts(String key, StateSpoutSpec val) { if (this.state_spouts == null) { this.state_spouts = new HashMap(); } this.state_spouts.put(key, val); } public Map get_state_spouts() { return this.state_spouts; } public void set_state_spouts(Map state_spouts) { this.state_spouts = state_spouts; } public void unset_state_spouts() { this.state_spouts = null; } /** Returns true if field state_spouts is set (has been assigned a value) and false otherwise */ public boolean is_set_state_spouts() { return this.state_spouts != null; } public void set_state_spouts_isSet(boolean value) { if (!value) { this.state_spouts = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case SPOUTS: if (value == null) { unset_spouts(); } else { set_spouts((Map)value); } break; case BOLTS: if (value == null) { unset_bolts(); } else { set_bolts((Map)value); } break; case STATE_SPOUTS: if (value == null) { unset_state_spouts(); } else { set_state_spouts((Map)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case SPOUTS: return get_spouts(); case BOLTS: return get_bolts(); case STATE_SPOUTS: return get_state_spouts(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case SPOUTS: return is_set_spouts(); case BOLTS: return is_set_bolts(); case STATE_SPOUTS: return is_set_state_spouts(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof StormTopology) return this.equals((StormTopology)that); return false; } public boolean equals(StormTopology that) { if (that == null) return false; boolean this_present_spouts = true && this.is_set_spouts(); boolean that_present_spouts = true && that.is_set_spouts(); if (this_present_spouts || that_present_spouts) { if (!(this_present_spouts && that_present_spouts)) return false; if (!this.spouts.equals(that.spouts)) return false; } boolean this_present_bolts = true && this.is_set_bolts(); boolean that_present_bolts = true && that.is_set_bolts(); if (this_present_bolts || that_present_bolts) { if (!(this_present_bolts && that_present_bolts)) return false; if (!this.bolts.equals(that.bolts)) return false; } boolean this_present_state_spouts = true && this.is_set_state_spouts(); boolean that_present_state_spouts = true && that.is_set_state_spouts(); if (this_present_state_spouts || that_present_state_spouts) { if (!(this_present_state_spouts && that_present_state_spouts)) return false; if (!this.state_spouts.equals(that.state_spouts)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_spouts = true && (is_set_spouts()); builder.append(present_spouts); if (present_spouts) builder.append(spouts); boolean present_bolts = true && (is_set_bolts()); builder.append(present_bolts); if (present_bolts) builder.append(bolts); boolean present_state_spouts = true && (is_set_state_spouts()); builder.append(present_state_spouts); if (present_state_spouts) builder.append(state_spouts); return builder.toHashCode(); } public int compareTo(StormTopology other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; StormTopology typedOther = (StormTopology)other; lastComparison = Boolean.valueOf(is_set_spouts()).compareTo(typedOther.is_set_spouts()); if (lastComparison != 0) { return lastComparison; } if (is_set_spouts()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.spouts, typedOther.spouts); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_bolts()).compareTo(typedOther.is_set_bolts()); if (lastComparison != 0) { return lastComparison; } if (is_set_bolts()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.bolts, typedOther.bolts); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_state_spouts()).compareTo(typedOther.is_set_state_spouts()); if (lastComparison != 0) { return lastComparison; } if (is_set_state_spouts()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.state_spouts, typedOther.state_spouts); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // SPOUTS if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map22 = iprot.readMapBegin(); this.spouts = new HashMap(2*_map22.size); for (int _i23 = 0; _i23 < _map22.size; ++_i23) { String _key24; // required SpoutSpec _val25; // required _key24 = iprot.readString(); _val25 = new SpoutSpec(); _val25.read(iprot); this.spouts.put(_key24, _val25); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // BOLTS if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map26 = iprot.readMapBegin(); this.bolts = new HashMap(2*_map26.size); for (int _i27 = 0; _i27 < _map26.size; ++_i27) { String _key28; // required Bolt _val29; // required _key28 = iprot.readString(); _val29 = new Bolt(); _val29.read(iprot); this.bolts.put(_key28, _val29); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // STATE_SPOUTS if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map30 = iprot.readMapBegin(); this.state_spouts = new HashMap(2*_map30.size); for (int _i31 = 0; _i31 < _map30.size; ++_i31) { String _key32; // required StateSpoutSpec _val33; // required _key32 = iprot.readString(); _val33 = new StateSpoutSpec(); _val33.read(iprot); this.state_spouts.put(_key32, _val33); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.spouts != null) { oprot.writeFieldBegin(SPOUTS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.spouts.size())); for (Map.Entry _iter34 : this.spouts.entrySet()) { oprot.writeString(_iter34.getKey()); _iter34.getValue().write(oprot); } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.bolts != null) { oprot.writeFieldBegin(BOLTS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.bolts.size())); for (Map.Entry _iter35 : this.bolts.entrySet()) { oprot.writeString(_iter35.getKey()); _iter35.getValue().write(oprot); } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } if (this.state_spouts != null) { oprot.writeFieldBegin(STATE_SPOUTS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.state_spouts.size())); for (Map.Entry _iter36 : this.state_spouts.entrySet()) { oprot.writeString(_iter36.getKey()); _iter36.getValue().write(oprot); } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("StormTopology("); boolean first = true; sb.append("spouts:"); if (this.spouts == null) { sb.append("null"); } else { sb.append(this.spouts); } first = false; if (!first) sb.append(", "); sb.append("bolts:"); if (this.bolts == null) { sb.append("null"); } else { sb.append(this.bolts); } first = false; if (!first) sb.append(", "); sb.append("state_spouts:"); if (this.state_spouts == null) { sb.append("null"); } else { sb.append(this.state_spouts); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_spouts()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'spouts' is unset! Struct:" + toString()); } if (!is_set_bolts()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'bolts' is unset! Struct:" + toString()); } if (!is_set_state_spouts()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'state_spouts' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/StreamInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class StreamInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("StreamInfo"); private static final org.apache.thrift7.protocol.TField OUTPUT_FIELDS_FIELD_DESC = new org.apache.thrift7.protocol.TField("output_fields", org.apache.thrift7.protocol.TType.LIST, (short)1); private static final org.apache.thrift7.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("direct", org.apache.thrift7.protocol.TType.BOOL, (short)2); private List output_fields; // required private boolean direct; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { OUTPUT_FIELDS((short)1, "output_fields"), DIRECT((short)2, "direct"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // OUTPUT_FIELDS return OUTPUT_FIELDS; case 2: // DIRECT return DIRECT; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __DIRECT_ISSET_ID = 0; private BitSet __isset_bit_vector = new BitSet(1); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.OUTPUT_FIELDS, new org.apache.thrift7.meta_data.FieldMetaData("output_fields", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING)))); tmpMap.put(_Fields.DIRECT, new org.apache.thrift7.meta_data.FieldMetaData("direct", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.BOOL))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(StreamInfo.class, metaDataMap); } public StreamInfo() { } public StreamInfo( List output_fields, boolean direct) { this(); this.output_fields = output_fields; this.direct = direct; set_direct_isSet(true); } /** * Performs a deep copy on other. */ public StreamInfo(StreamInfo other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_output_fields()) { List __this__output_fields = new ArrayList(); for (String other_element : other.output_fields) { __this__output_fields.add(other_element); } this.output_fields = __this__output_fields; } this.direct = other.direct; } public StreamInfo deepCopy() { return new StreamInfo(this); } @Override public void clear() { this.output_fields = null; set_direct_isSet(false); this.direct = false; } public int get_output_fields_size() { return (this.output_fields == null) ? 0 : this.output_fields.size(); } public java.util.Iterator get_output_fields_iterator() { return (this.output_fields == null) ? null : this.output_fields.iterator(); } public void add_to_output_fields(String elem) { if (this.output_fields == null) { this.output_fields = new ArrayList(); } this.output_fields.add(elem); } public List get_output_fields() { return this.output_fields; } public void set_output_fields(List output_fields) { this.output_fields = output_fields; } public void unset_output_fields() { this.output_fields = null; } /** Returns true if field output_fields is set (has been assigned a value) and false otherwise */ public boolean is_set_output_fields() { return this.output_fields != null; } public void set_output_fields_isSet(boolean value) { if (!value) { this.output_fields = null; } } public boolean is_direct() { return this.direct; } public void set_direct(boolean direct) { this.direct = direct; set_direct_isSet(true); } public void unset_direct() { __isset_bit_vector.clear(__DIRECT_ISSET_ID); } /** Returns true if field direct is set (has been assigned a value) and false otherwise */ public boolean is_set_direct() { return __isset_bit_vector.get(__DIRECT_ISSET_ID); } public void set_direct_isSet(boolean value) { __isset_bit_vector.set(__DIRECT_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { switch (field) { case OUTPUT_FIELDS: if (value == null) { unset_output_fields(); } else { set_output_fields((List)value); } break; case DIRECT: if (value == null) { unset_direct(); } else { set_direct((Boolean)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case OUTPUT_FIELDS: return get_output_fields(); case DIRECT: return Boolean.valueOf(is_direct()); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case OUTPUT_FIELDS: return is_set_output_fields(); case DIRECT: return is_set_direct(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof StreamInfo) return this.equals((StreamInfo)that); return false; } public boolean equals(StreamInfo that) { if (that == null) return false; boolean this_present_output_fields = true && this.is_set_output_fields(); boolean that_present_output_fields = true && that.is_set_output_fields(); if (this_present_output_fields || that_present_output_fields) { if (!(this_present_output_fields && that_present_output_fields)) return false; if (!this.output_fields.equals(that.output_fields)) return false; } boolean this_present_direct = true; boolean that_present_direct = true; if (this_present_direct || that_present_direct) { if (!(this_present_direct && that_present_direct)) return false; if (this.direct != that.direct) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_output_fields = true && (is_set_output_fields()); builder.append(present_output_fields); if (present_output_fields) builder.append(output_fields); boolean present_direct = true; builder.append(present_direct); if (present_direct) builder.append(direct); return builder.toHashCode(); } public int compareTo(StreamInfo other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; StreamInfo typedOther = (StreamInfo)other; lastComparison = Boolean.valueOf(is_set_output_fields()).compareTo(typedOther.is_set_output_fields()); if (lastComparison != 0) { return lastComparison; } if (is_set_output_fields()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.output_fields, typedOther.output_fields); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_direct()).compareTo(typedOther.is_set_direct()); if (lastComparison != 0) { return lastComparison; } if (is_set_direct()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.direct, typedOther.direct); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // OUTPUT_FIELDS if (field.type == org.apache.thrift7.protocol.TType.LIST) { { org.apache.thrift7.protocol.TList _list8 = iprot.readListBegin(); this.output_fields = new ArrayList(_list8.size); for (int _i9 = 0; _i9 < _list8.size; ++_i9) { String _elem10; // required _elem10 = iprot.readString(); this.output_fields.add(_elem10); } iprot.readListEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // DIRECT if (field.type == org.apache.thrift7.protocol.TType.BOOL) { this.direct = iprot.readBool(); set_direct_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.output_fields != null) { oprot.writeFieldBegin(OUTPUT_FIELDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRING, this.output_fields.size())); for (String _iter11 : this.output_fields) { oprot.writeString(_iter11); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldBegin(DIRECT_FIELD_DESC); oprot.writeBool(this.direct); oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("StreamInfo("); boolean first = true; sb.append("output_fields:"); if (this.output_fields == null) { sb.append("null"); } else { sb.append(this.output_fields); } first = false; if (!first) sb.append(", "); sb.append("direct:"); sb.append(this.direct); first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_output_fields()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'output_fields' is unset! Struct:" + toString()); } if (!is_set_direct()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'direct' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SubmitOptions implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SubmitOptions"); private static final org.apache.thrift7.protocol.TField INITIAL_STATUS_FIELD_DESC = new org.apache.thrift7.protocol.TField("initial_status", org.apache.thrift7.protocol.TType.I32, (short)1); private TopologyInitialStatus initial_status; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { /** * * @see TopologyInitialStatus */ INITIAL_STATUS((short)1, "initial_status"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // INITIAL_STATUS return INITIAL_STATUS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.INITIAL_STATUS, new org.apache.thrift7.meta_data.FieldMetaData("initial_status", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.EnumMetaData(org.apache.thrift7.protocol.TType.ENUM, TopologyInitialStatus.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SubmitOptions.class, metaDataMap); } public SubmitOptions() { } public SubmitOptions( TopologyInitialStatus initial_status) { this(); this.initial_status = initial_status; } /** * Performs a deep copy on other. */ public SubmitOptions(SubmitOptions other) { if (other.is_set_initial_status()) { this.initial_status = other.initial_status; } } public SubmitOptions deepCopy() { return new SubmitOptions(this); } @Override public void clear() { this.initial_status = null; } /** * * @see TopologyInitialStatus */ public TopologyInitialStatus get_initial_status() { return this.initial_status; } /** * * @see TopologyInitialStatus */ public void set_initial_status(TopologyInitialStatus initial_status) { this.initial_status = initial_status; } public void unset_initial_status() { this.initial_status = null; } /** Returns true if field initial_status is set (has been assigned a value) and false otherwise */ public boolean is_set_initial_status() { return this.initial_status != null; } public void set_initial_status_isSet(boolean value) { if (!value) { this.initial_status = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case INITIAL_STATUS: if (value == null) { unset_initial_status(); } else { set_initial_status((TopologyInitialStatus)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case INITIAL_STATUS: return get_initial_status(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case INITIAL_STATUS: return is_set_initial_status(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof SubmitOptions) return this.equals((SubmitOptions)that); return false; } public boolean equals(SubmitOptions that) { if (that == null) return false; boolean this_present_initial_status = true && this.is_set_initial_status(); boolean that_present_initial_status = true && that.is_set_initial_status(); if (this_present_initial_status || that_present_initial_status) { if (!(this_present_initial_status && that_present_initial_status)) return false; if (!this.initial_status.equals(that.initial_status)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_initial_status = true && (is_set_initial_status()); builder.append(present_initial_status); if (present_initial_status) builder.append(initial_status.getValue()); return builder.toHashCode(); } public int compareTo(SubmitOptions other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; SubmitOptions typedOther = (SubmitOptions)other; lastComparison = Boolean.valueOf(is_set_initial_status()).compareTo(typedOther.is_set_initial_status()); if (lastComparison != 0) { return lastComparison; } if (is_set_initial_status()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.initial_status, typedOther.initial_status); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // INITIAL_STATUS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.initial_status = TopologyInitialStatus.findByValue(iprot.readI32()); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.initial_status != null) { oprot.writeFieldBegin(INITIAL_STATUS_FIELD_DESC); oprot.writeI32(this.initial_status.getValue()); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("SubmitOptions("); boolean first = true; sb.append("initial_status:"); if (this.initial_status == null) { sb.append("null"); } else { sb.append(this.initial_status); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_initial_status()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'initial_status' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SupervisorSummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SupervisorSummary"); private static final org.apache.thrift7.protocol.TField HOST_FIELD_DESC = new org.apache.thrift7.protocol.TField("host", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)2); private static final org.apache.thrift7.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_workers", org.apache.thrift7.protocol.TType.I32, (short)3); private static final org.apache.thrift7.protocol.TField NUM_USED_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_used_workers", org.apache.thrift7.protocol.TType.I32, (short)4); private static final org.apache.thrift7.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("supervisor_id", org.apache.thrift7.protocol.TType.STRING, (short)5); private String host; // required private int uptime_secs; // required private int num_workers; // required private int num_used_workers; // required private String supervisor_id; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { HOST((short)1, "host"), UPTIME_SECS((short)2, "uptime_secs"), NUM_WORKERS((short)3, "num_workers"), NUM_USED_WORKERS((short)4, "num_used_workers"), SUPERVISOR_ID((short)5, "supervisor_id"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // HOST return HOST; case 2: // UPTIME_SECS return UPTIME_SECS; case 3: // NUM_WORKERS return NUM_WORKERS; case 4: // NUM_USED_WORKERS return NUM_USED_WORKERS; case 5: // SUPERVISOR_ID return SUPERVISOR_ID; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __UPTIME_SECS_ISSET_ID = 0; private static final int __NUM_WORKERS_ISSET_ID = 1; private static final int __NUM_USED_WORKERS_ISSET_ID = 2; private BitSet __isset_bit_vector = new BitSet(3); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.HOST, new org.apache.thrift7.meta_data.FieldMetaData("host", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.NUM_USED_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_used_workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift7.meta_data.FieldMetaData("supervisor_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap); } public SupervisorSummary() { } public SupervisorSummary( String host, int uptime_secs, int num_workers, int num_used_workers, String supervisor_id) { this(); this.host = host; this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); this.num_workers = num_workers; set_num_workers_isSet(true); this.num_used_workers = num_used_workers; set_num_used_workers_isSet(true); this.supervisor_id = supervisor_id; } /** * Performs a deep copy on other. */ public SupervisorSummary(SupervisorSummary other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_host()) { this.host = other.host; } this.uptime_secs = other.uptime_secs; this.num_workers = other.num_workers; this.num_used_workers = other.num_used_workers; if (other.is_set_supervisor_id()) { this.supervisor_id = other.supervisor_id; } } public SupervisorSummary deepCopy() { return new SupervisorSummary(this); } @Override public void clear() { this.host = null; set_uptime_secs_isSet(false); this.uptime_secs = 0; set_num_workers_isSet(false); this.num_workers = 0; set_num_used_workers_isSet(false); this.num_used_workers = 0; this.supervisor_id = null; } public String get_host() { return this.host; } public void set_host(String host) { this.host = host; } public void unset_host() { this.host = null; } /** Returns true if field host is set (has been assigned a value) and false otherwise */ public boolean is_set_host() { return this.host != null; } public void set_host_isSet(boolean value) { if (!value) { this.host = null; } } public int get_uptime_secs() { return this.uptime_secs; } public void set_uptime_secs(int uptime_secs) { this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); } public void unset_uptime_secs() { __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); } /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_uptime_secs() { return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); } public void set_uptime_secs_isSet(boolean value) { __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); } public int get_num_workers() { return this.num_workers; } public void set_num_workers(int num_workers) { this.num_workers = num_workers; set_num_workers_isSet(true); } public void unset_num_workers() { __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID); } /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ public boolean is_set_num_workers() { return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID); } public void set_num_workers_isSet(boolean value) { __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value); } public int get_num_used_workers() { return this.num_used_workers; } public void set_num_used_workers(int num_used_workers) { this.num_used_workers = num_used_workers; set_num_used_workers_isSet(true); } public void unset_num_used_workers() { __isset_bit_vector.clear(__NUM_USED_WORKERS_ISSET_ID); } /** Returns true if field num_used_workers is set (has been assigned a value) and false otherwise */ public boolean is_set_num_used_workers() { return __isset_bit_vector.get(__NUM_USED_WORKERS_ISSET_ID); } public void set_num_used_workers_isSet(boolean value) { __isset_bit_vector.set(__NUM_USED_WORKERS_ISSET_ID, value); } public String get_supervisor_id() { return this.supervisor_id; } public void set_supervisor_id(String supervisor_id) { this.supervisor_id = supervisor_id; } public void unset_supervisor_id() { this.supervisor_id = null; } /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */ public boolean is_set_supervisor_id() { return this.supervisor_id != null; } public void set_supervisor_id_isSet(boolean value) { if (!value) { this.supervisor_id = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case HOST: if (value == null) { unset_host(); } else { set_host((String)value); } break; case UPTIME_SECS: if (value == null) { unset_uptime_secs(); } else { set_uptime_secs((Integer)value); } break; case NUM_WORKERS: if (value == null) { unset_num_workers(); } else { set_num_workers((Integer)value); } break; case NUM_USED_WORKERS: if (value == null) { unset_num_used_workers(); } else { set_num_used_workers((Integer)value); } break; case SUPERVISOR_ID: if (value == null) { unset_supervisor_id(); } else { set_supervisor_id((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case HOST: return get_host(); case UPTIME_SECS: return Integer.valueOf(get_uptime_secs()); case NUM_WORKERS: return Integer.valueOf(get_num_workers()); case NUM_USED_WORKERS: return Integer.valueOf(get_num_used_workers()); case SUPERVISOR_ID: return get_supervisor_id(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case HOST: return is_set_host(); case UPTIME_SECS: return is_set_uptime_secs(); case NUM_WORKERS: return is_set_num_workers(); case NUM_USED_WORKERS: return is_set_num_used_workers(); case SUPERVISOR_ID: return is_set_supervisor_id(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof SupervisorSummary) return this.equals((SupervisorSummary)that); return false; } public boolean equals(SupervisorSummary that) { if (that == null) return false; boolean this_present_host = true && this.is_set_host(); boolean that_present_host = true && that.is_set_host(); if (this_present_host || that_present_host) { if (!(this_present_host && that_present_host)) return false; if (!this.host.equals(that.host)) return false; } boolean this_present_uptime_secs = true; boolean that_present_uptime_secs = true; if (this_present_uptime_secs || that_present_uptime_secs) { if (!(this_present_uptime_secs && that_present_uptime_secs)) return false; if (this.uptime_secs != that.uptime_secs) return false; } boolean this_present_num_workers = true; boolean that_present_num_workers = true; if (this_present_num_workers || that_present_num_workers) { if (!(this_present_num_workers && that_present_num_workers)) return false; if (this.num_workers != that.num_workers) return false; } boolean this_present_num_used_workers = true; boolean that_present_num_used_workers = true; if (this_present_num_used_workers || that_present_num_used_workers) { if (!(this_present_num_used_workers && that_present_num_used_workers)) return false; if (this.num_used_workers != that.num_used_workers) return false; } boolean this_present_supervisor_id = true && this.is_set_supervisor_id(); boolean that_present_supervisor_id = true && that.is_set_supervisor_id(); if (this_present_supervisor_id || that_present_supervisor_id) { if (!(this_present_supervisor_id && that_present_supervisor_id)) return false; if (!this.supervisor_id.equals(that.supervisor_id)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_host = true && (is_set_host()); builder.append(present_host); if (present_host) builder.append(host); boolean present_uptime_secs = true; builder.append(present_uptime_secs); if (present_uptime_secs) builder.append(uptime_secs); boolean present_num_workers = true; builder.append(present_num_workers); if (present_num_workers) builder.append(num_workers); boolean present_num_used_workers = true; builder.append(present_num_used_workers); if (present_num_used_workers) builder.append(num_used_workers); boolean present_supervisor_id = true && (is_set_supervisor_id()); builder.append(present_supervisor_id); if (present_supervisor_id) builder.append(supervisor_id); return builder.toHashCode(); } public int compareTo(SupervisorSummary other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; SupervisorSummary typedOther = (SupervisorSummary)other; lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host()); if (lastComparison != 0) { return lastComparison; } if (is_set_host()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.host, typedOther.host); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_uptime_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers()); if (lastComparison != 0) { return lastComparison; } if (is_set_num_workers()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_num_used_workers()).compareTo(typedOther.is_set_num_used_workers()); if (lastComparison != 0) { return lastComparison; } if (is_set_num_used_workers()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_used_workers, typedOther.num_used_workers); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(typedOther.is_set_supervisor_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_supervisor_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.supervisor_id, typedOther.supervisor_id); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // HOST if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.host = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // UPTIME_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.uptime_secs = iprot.readI32(); set_uptime_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // NUM_WORKERS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.num_workers = iprot.readI32(); set_num_workers_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // NUM_USED_WORKERS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.num_used_workers = iprot.readI32(); set_num_used_workers_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 5: // SUPERVISOR_ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.supervisor_id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.host != null) { oprot.writeFieldBegin(HOST_FIELD_DESC); oprot.writeString(this.host); oprot.writeFieldEnd(); } oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); oprot.writeI32(this.uptime_secs); oprot.writeFieldEnd(); oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); oprot.writeI32(this.num_workers); oprot.writeFieldEnd(); oprot.writeFieldBegin(NUM_USED_WORKERS_FIELD_DESC); oprot.writeI32(this.num_used_workers); oprot.writeFieldEnd(); if (this.supervisor_id != null) { oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC); oprot.writeString(this.supervisor_id); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("SupervisorSummary("); boolean first = true; sb.append("host:"); if (this.host == null) { sb.append("null"); } else { sb.append(this.host); } first = false; if (!first) sb.append(", "); sb.append("uptime_secs:"); sb.append(this.uptime_secs); first = false; if (!first) sb.append(", "); sb.append("num_workers:"); sb.append(this.num_workers); first = false; if (!first) sb.append(", "); sb.append("num_used_workers:"); sb.append(this.num_used_workers); first = false; if (!first) sb.append(", "); sb.append("supervisor_id:"); if (this.supervisor_id == null) { sb.append("null"); } else { sb.append(this.supervisor_id); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_host()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString()); } if (!is_set_uptime_secs()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); } if (!is_set_num_workers()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString()); } if (!is_set_num_used_workers()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_used_workers' is unset! Struct:" + toString()); } if (!is_set_supervisor_id()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'supervisor_id' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TopologyInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TopologyInfo"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)2); private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)3); private static final org.apache.thrift7.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift7.protocol.TField("executors", org.apache.thrift7.protocol.TType.LIST, (short)4); private static final org.apache.thrift7.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift7.protocol.TField("status", org.apache.thrift7.protocol.TType.STRING, (short)5); private static final org.apache.thrift7.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift7.protocol.TField("errors", org.apache.thrift7.protocol.TType.MAP, (short)6); private String id; // required private String name; // required private int uptime_secs; // required private List executors; // required private String status; // required private Map> errors; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"), NAME((short)2, "name"), UPTIME_SECS((short)3, "uptime_secs"), EXECUTORS((short)4, "executors"), STATUS((short)5, "status"), ERRORS((short)6, "errors"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; case 2: // NAME return NAME; case 3: // UPTIME_SECS return UPTIME_SECS; case 4: // EXECUTORS return EXECUTORS; case 5: // STATUS return STATUS; case 6: // ERRORS return ERRORS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __UPTIME_SECS_ISSET_ID = 0; private BitSet __isset_bit_vector = new BitSet(1); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.EXECUTORS, new org.apache.thrift7.meta_data.FieldMetaData("executors", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ExecutorSummary.class)))); tmpMap.put(_Fields.STATUS, new org.apache.thrift7.meta_data.FieldMetaData("status", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.ERRORS, new org.apache.thrift7.meta_data.FieldMetaData("errors", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ErrorInfo.class))))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap); } public TopologyInfo() { } public TopologyInfo( String id, String name, int uptime_secs, List executors, String status, Map> errors) { this(); this.id = id; this.name = name; this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); this.executors = executors; this.status = status; this.errors = errors; } /** * Performs a deep copy on other. */ public TopologyInfo(TopologyInfo other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_id()) { this.id = other.id; } if (other.is_set_name()) { this.name = other.name; } this.uptime_secs = other.uptime_secs; if (other.is_set_executors()) { List __this__executors = new ArrayList(); for (ExecutorSummary other_element : other.executors) { __this__executors.add(new ExecutorSummary(other_element)); } this.executors = __this__executors; } if (other.is_set_status()) { this.status = other.status; } if (other.is_set_errors()) { Map> __this__errors = new HashMap>(); for (Map.Entry> other_element : other.errors.entrySet()) { String other_element_key = other_element.getKey(); List other_element_value = other_element.getValue(); String __this__errors_copy_key = other_element_key; List __this__errors_copy_value = new ArrayList(); for (ErrorInfo other_element_value_element : other_element_value) { __this__errors_copy_value.add(new ErrorInfo(other_element_value_element)); } __this__errors.put(__this__errors_copy_key, __this__errors_copy_value); } this.errors = __this__errors; } } public TopologyInfo deepCopy() { return new TopologyInfo(this); } @Override public void clear() { this.id = null; this.name = null; set_uptime_secs_isSet(false); this.uptime_secs = 0; this.executors = null; this.status = null; this.errors = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public int get_uptime_secs() { return this.uptime_secs; } public void set_uptime_secs(int uptime_secs) { this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); } public void unset_uptime_secs() { __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); } /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_uptime_secs() { return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); } public void set_uptime_secs_isSet(boolean value) { __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); } public int get_executors_size() { return (this.executors == null) ? 0 : this.executors.size(); } public java.util.Iterator get_executors_iterator() { return (this.executors == null) ? null : this.executors.iterator(); } public void add_to_executors(ExecutorSummary elem) { if (this.executors == null) { this.executors = new ArrayList(); } this.executors.add(elem); } public List get_executors() { return this.executors; } public void set_executors(List executors) { this.executors = executors; } public void unset_executors() { this.executors = null; } /** Returns true if field executors is set (has been assigned a value) and false otherwise */ public boolean is_set_executors() { return this.executors != null; } public void set_executors_isSet(boolean value) { if (!value) { this.executors = null; } } public String get_status() { return this.status; } public void set_status(String status) { this.status = status; } public void unset_status() { this.status = null; } /** Returns true if field status is set (has been assigned a value) and false otherwise */ public boolean is_set_status() { return this.status != null; } public void set_status_isSet(boolean value) { if (!value) { this.status = null; } } public int get_errors_size() { return (this.errors == null) ? 0 : this.errors.size(); } public void put_to_errors(String key, List val) { if (this.errors == null) { this.errors = new HashMap>(); } this.errors.put(key, val); } public Map> get_errors() { return this.errors; } public void set_errors(Map> errors) { this.errors = errors; } public void unset_errors() { this.errors = null; } /** Returns true if field errors is set (has been assigned a value) and false otherwise */ public boolean is_set_errors() { return this.errors != null; } public void set_errors_isSet(boolean value) { if (!value) { this.errors = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; case UPTIME_SECS: if (value == null) { unset_uptime_secs(); } else { set_uptime_secs((Integer)value); } break; case EXECUTORS: if (value == null) { unset_executors(); } else { set_executors((List)value); } break; case STATUS: if (value == null) { unset_status(); } else { set_status((String)value); } break; case ERRORS: if (value == null) { unset_errors(); } else { set_errors((Map>)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); case NAME: return get_name(); case UPTIME_SECS: return Integer.valueOf(get_uptime_secs()); case EXECUTORS: return get_executors(); case STATUS: return get_status(); case ERRORS: return get_errors(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); case NAME: return is_set_name(); case UPTIME_SECS: return is_set_uptime_secs(); case EXECUTORS: return is_set_executors(); case STATUS: return is_set_status(); case ERRORS: return is_set_errors(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof TopologyInfo) return this.equals((TopologyInfo)that); return false; } public boolean equals(TopologyInfo that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } boolean this_present_uptime_secs = true; boolean that_present_uptime_secs = true; if (this_present_uptime_secs || that_present_uptime_secs) { if (!(this_present_uptime_secs && that_present_uptime_secs)) return false; if (this.uptime_secs != that.uptime_secs) return false; } boolean this_present_executors = true && this.is_set_executors(); boolean that_present_executors = true && that.is_set_executors(); if (this_present_executors || that_present_executors) { if (!(this_present_executors && that_present_executors)) return false; if (!this.executors.equals(that.executors)) return false; } boolean this_present_status = true && this.is_set_status(); boolean that_present_status = true && that.is_set_status(); if (this_present_status || that_present_status) { if (!(this_present_status && that_present_status)) return false; if (!this.status.equals(that.status)) return false; } boolean this_present_errors = true && this.is_set_errors(); boolean that_present_errors = true && that.is_set_errors(); if (this_present_errors || that_present_errors) { if (!(this_present_errors && that_present_errors)) return false; if (!this.errors.equals(that.errors)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); boolean present_uptime_secs = true; builder.append(present_uptime_secs); if (present_uptime_secs) builder.append(uptime_secs); boolean present_executors = true && (is_set_executors()); builder.append(present_executors); if (present_executors) builder.append(executors); boolean present_status = true && (is_set_status()); builder.append(present_status); if (present_status) builder.append(status); boolean present_errors = true && (is_set_errors()); builder.append(present_errors); if (present_errors) builder.append(errors); return builder.toHashCode(); } public int compareTo(TopologyInfo other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; TopologyInfo typedOther = (TopologyInfo)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_uptime_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_executors()).compareTo(typedOther.is_set_executors()); if (lastComparison != 0) { return lastComparison; } if (is_set_executors()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.executors, typedOther.executors); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_status()).compareTo(typedOther.is_set_status()); if (lastComparison != 0) { return lastComparison; } if (is_set_status()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.status, typedOther.status); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_errors()).compareTo(typedOther.is_set_errors()); if (lastComparison != 0) { return lastComparison; } if (is_set_errors()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.errors, typedOther.errors); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // UPTIME_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.uptime_secs = iprot.readI32(); set_uptime_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // EXECUTORS if (field.type == org.apache.thrift7.protocol.TType.LIST) { { org.apache.thrift7.protocol.TList _list145 = iprot.readListBegin(); this.executors = new ArrayList(_list145.size); for (int _i146 = 0; _i146 < _list145.size; ++_i146) { ExecutorSummary _elem147; // required _elem147 = new ExecutorSummary(); _elem147.read(iprot); this.executors.add(_elem147); } iprot.readListEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 5: // STATUS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.status = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 6: // ERRORS if (field.type == org.apache.thrift7.protocol.TType.MAP) { { org.apache.thrift7.protocol.TMap _map148 = iprot.readMapBegin(); this.errors = new HashMap>(2*_map148.size); for (int _i149 = 0; _i149 < _map148.size; ++_i149) { String _key150; // required List _val151; // required _key150 = iprot.readString(); { org.apache.thrift7.protocol.TList _list152 = iprot.readListBegin(); _val151 = new ArrayList(_list152.size); for (int _i153 = 0; _i153 < _list152.size; ++_i153) { ErrorInfo _elem154; // required _elem154 = new ErrorInfo(); _elem154.read(iprot); _val151.add(_elem154); } iprot.readListEnd(); } this.errors.put(_key150, _val151); } iprot.readMapEnd(); } } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); oprot.writeI32(this.uptime_secs); oprot.writeFieldEnd(); if (this.executors != null) { oprot.writeFieldBegin(EXECUTORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.executors.size())); for (ExecutorSummary _iter155 : this.executors) { _iter155.write(oprot); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } if (this.status != null) { oprot.writeFieldBegin(STATUS_FIELD_DESC); oprot.writeString(this.status); oprot.writeFieldEnd(); } if (this.errors != null) { oprot.writeFieldBegin(ERRORS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.LIST, this.errors.size())); for (Map.Entry> _iter156 : this.errors.entrySet()) { oprot.writeString(_iter156.getKey()); { oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, _iter156.getValue().size())); for (ErrorInfo _iter157 : _iter156.getValue()) { _iter157.write(oprot); } oprot.writeListEnd(); } } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("TopologyInfo("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; if (!first) sb.append(", "); sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; if (!first) sb.append(", "); sb.append("uptime_secs:"); sb.append(this.uptime_secs); first = false; if (!first) sb.append(", "); sb.append("executors:"); if (this.executors == null) { sb.append("null"); } else { sb.append(this.executors); } first = false; if (!first) sb.append(", "); sb.append("status:"); if (this.status == null) { sb.append("null"); } else { sb.append(this.status); } first = false; if (!first) sb.append(", "); sb.append("errors:"); if (this.errors == null) { sb.append("null"); } else { sb.append(this.errors); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_id()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString()); } if (!is_set_name()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); } if (!is_set_uptime_secs()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); } if (!is_set_executors()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'executors' is unset! Struct:" + toString()); } if (!is_set_status()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); } if (!is_set_errors()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'errors' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/TopologyInitialStatus.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import java.util.Map; import java.util.HashMap; import org.apache.thrift7.TEnum; public enum TopologyInitialStatus implements org.apache.thrift7.TEnum { ACTIVE(1), INACTIVE(2); private final int value; private TopologyInitialStatus(int value) { this.value = value; } /** * Get the integer value of this enum value, as defined in the Thrift IDL. */ public int getValue() { return value; } /** * Find a the enum type by its integer value, as defined in the Thrift IDL. * @return null if the value is not found. */ public static TopologyInitialStatus findByValue(int value) { switch (value) { case 1: return ACTIVE; case 2: return INACTIVE; default: return null; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/generated/TopologySummary.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Autogenerated by Thrift Compiler (0.7.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING */ package backtype.storm.generated; import org.apache.commons.lang.builder.HashCodeBuilder; import java.util.List; import java.util.ArrayList; import java.util.Map; import java.util.HashMap; import java.util.EnumMap; import java.util.Set; import java.util.HashSet; import java.util.EnumSet; import java.util.Collections; import java.util.BitSet; import java.nio.ByteBuffer; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TopologySummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TopologySummary"); private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)2); private static final org.apache.thrift7.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_tasks", org.apache.thrift7.protocol.TType.I32, (short)3); private static final org.apache.thrift7.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_executors", org.apache.thrift7.protocol.TType.I32, (short)4); private static final org.apache.thrift7.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_workers", org.apache.thrift7.protocol.TType.I32, (short)5); private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)6); private static final org.apache.thrift7.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift7.protocol.TField("status", org.apache.thrift7.protocol.TType.STRING, (short)7); private String id; // required private String name; // required private int num_tasks; // required private int num_executors; // required private int num_workers; // required private int uptime_secs; // required private String status; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift7.TFieldIdEnum { ID((short)1, "id"), NAME((short)2, "name"), NUM_TASKS((short)3, "num_tasks"), NUM_EXECUTORS((short)4, "num_executors"), NUM_WORKERS((short)5, "num_workers"), UPTIME_SECS((short)6, "uptime_secs"), STATUS((short)7, "status"); private static final Map byName = new HashMap(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { byName.put(field.getFieldName(), field); } } /** * Find the _Fields constant that matches fieldId, or null if its not found. */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // ID return ID; case 2: // NAME return NAME; case 3: // NUM_TASKS return NUM_TASKS; case 4: // NUM_EXECUTORS return NUM_EXECUTORS; case 5: // NUM_WORKERS return NUM_WORKERS; case 6: // UPTIME_SECS return UPTIME_SECS; case 7: // STATUS return STATUS; default: return null; } } /** * Find the _Fields constant that matches fieldId, throwing an exception * if it is not found. */ public static _Fields findByThriftIdOrThrow(int fieldId) { _Fields fields = findByThriftId(fieldId); if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); return fields; } /** * Find the _Fields constant that matches name, or null if its not found. */ public static _Fields findByName(String name) { return byName.get(name); } private final short _thriftId; private final String _fieldName; _Fields(short thriftId, String fieldName) { _thriftId = thriftId; _fieldName = fieldName; } public short getThriftFieldId() { return _thriftId; } public String getFieldName() { return _fieldName; } } // isset id assignments private static final int __NUM_TASKS_ISSET_ID = 0; private static final int __NUM_EXECUTORS_ISSET_ID = 1; private static final int __NUM_WORKERS_ISSET_ID = 2; private static final int __UPTIME_SECS_ISSET_ID = 3; private BitSet __isset_bit_vector = new BitSet(4); public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift7.meta_data.FieldMetaData("num_tasks", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift7.meta_data.FieldMetaData("num_executors", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); tmpMap.put(_Fields.STATUS, new org.apache.thrift7.meta_data.FieldMetaData("status", org.apache.thrift7.TFieldRequirementType.REQUIRED, new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap); } public TopologySummary() { } public TopologySummary( String id, String name, int num_tasks, int num_executors, int num_workers, int uptime_secs, String status) { this(); this.id = id; this.name = name; this.num_tasks = num_tasks; set_num_tasks_isSet(true); this.num_executors = num_executors; set_num_executors_isSet(true); this.num_workers = num_workers; set_num_workers_isSet(true); this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); this.status = status; } /** * Performs a deep copy on other. */ public TopologySummary(TopologySummary other) { __isset_bit_vector.clear(); __isset_bit_vector.or(other.__isset_bit_vector); if (other.is_set_id()) { this.id = other.id; } if (other.is_set_name()) { this.name = other.name; } this.num_tasks = other.num_tasks; this.num_executors = other.num_executors; this.num_workers = other.num_workers; this.uptime_secs = other.uptime_secs; if (other.is_set_status()) { this.status = other.status; } } public TopologySummary deepCopy() { return new TopologySummary(this); } @Override public void clear() { this.id = null; this.name = null; set_num_tasks_isSet(false); this.num_tasks = 0; set_num_executors_isSet(false); this.num_executors = 0; set_num_workers_isSet(false); this.num_workers = 0; set_uptime_secs_isSet(false); this.uptime_secs = 0; this.status = null; } public String get_id() { return this.id; } public void set_id(String id) { this.id = id; } public void unset_id() { this.id = null; } /** Returns true if field id is set (has been assigned a value) and false otherwise */ public boolean is_set_id() { return this.id != null; } public void set_id_isSet(boolean value) { if (!value) { this.id = null; } } public String get_name() { return this.name; } public void set_name(String name) { this.name = name; } public void unset_name() { this.name = null; } /** Returns true if field name is set (has been assigned a value) and false otherwise */ public boolean is_set_name() { return this.name != null; } public void set_name_isSet(boolean value) { if (!value) { this.name = null; } } public int get_num_tasks() { return this.num_tasks; } public void set_num_tasks(int num_tasks) { this.num_tasks = num_tasks; set_num_tasks_isSet(true); } public void unset_num_tasks() { __isset_bit_vector.clear(__NUM_TASKS_ISSET_ID); } /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */ public boolean is_set_num_tasks() { return __isset_bit_vector.get(__NUM_TASKS_ISSET_ID); } public void set_num_tasks_isSet(boolean value) { __isset_bit_vector.set(__NUM_TASKS_ISSET_ID, value); } public int get_num_executors() { return this.num_executors; } public void set_num_executors(int num_executors) { this.num_executors = num_executors; set_num_executors_isSet(true); } public void unset_num_executors() { __isset_bit_vector.clear(__NUM_EXECUTORS_ISSET_ID); } /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */ public boolean is_set_num_executors() { return __isset_bit_vector.get(__NUM_EXECUTORS_ISSET_ID); } public void set_num_executors_isSet(boolean value) { __isset_bit_vector.set(__NUM_EXECUTORS_ISSET_ID, value); } public int get_num_workers() { return this.num_workers; } public void set_num_workers(int num_workers) { this.num_workers = num_workers; set_num_workers_isSet(true); } public void unset_num_workers() { __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID); } /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ public boolean is_set_num_workers() { return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID); } public void set_num_workers_isSet(boolean value) { __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value); } public int get_uptime_secs() { return this.uptime_secs; } public void set_uptime_secs(int uptime_secs) { this.uptime_secs = uptime_secs; set_uptime_secs_isSet(true); } public void unset_uptime_secs() { __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); } /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ public boolean is_set_uptime_secs() { return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); } public void set_uptime_secs_isSet(boolean value) { __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); } public String get_status() { return this.status; } public void set_status(String status) { this.status = status; } public void unset_status() { this.status = null; } /** Returns true if field status is set (has been assigned a value) and false otherwise */ public boolean is_set_status() { return this.status != null; } public void set_status_isSet(boolean value) { if (!value) { this.status = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: if (value == null) { unset_id(); } else { set_id((String)value); } break; case NAME: if (value == null) { unset_name(); } else { set_name((String)value); } break; case NUM_TASKS: if (value == null) { unset_num_tasks(); } else { set_num_tasks((Integer)value); } break; case NUM_EXECUTORS: if (value == null) { unset_num_executors(); } else { set_num_executors((Integer)value); } break; case NUM_WORKERS: if (value == null) { unset_num_workers(); } else { set_num_workers((Integer)value); } break; case UPTIME_SECS: if (value == null) { unset_uptime_secs(); } else { set_uptime_secs((Integer)value); } break; case STATUS: if (value == null) { unset_status(); } else { set_status((String)value); } break; } } public Object getFieldValue(_Fields field) { switch (field) { case ID: return get_id(); case NAME: return get_name(); case NUM_TASKS: return Integer.valueOf(get_num_tasks()); case NUM_EXECUTORS: return Integer.valueOf(get_num_executors()); case NUM_WORKERS: return Integer.valueOf(get_num_workers()); case UPTIME_SECS: return Integer.valueOf(get_uptime_secs()); case STATUS: return get_status(); } throw new IllegalStateException(); } /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ public boolean isSet(_Fields field) { if (field == null) { throw new IllegalArgumentException(); } switch (field) { case ID: return is_set_id(); case NAME: return is_set_name(); case NUM_TASKS: return is_set_num_tasks(); case NUM_EXECUTORS: return is_set_num_executors(); case NUM_WORKERS: return is_set_num_workers(); case UPTIME_SECS: return is_set_uptime_secs(); case STATUS: return is_set_status(); } throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; if (that instanceof TopologySummary) return this.equals((TopologySummary)that); return false; } public boolean equals(TopologySummary that) { if (that == null) return false; boolean this_present_id = true && this.is_set_id(); boolean that_present_id = true && that.is_set_id(); if (this_present_id || that_present_id) { if (!(this_present_id && that_present_id)) return false; if (!this.id.equals(that.id)) return false; } boolean this_present_name = true && this.is_set_name(); boolean that_present_name = true && that.is_set_name(); if (this_present_name || that_present_name) { if (!(this_present_name && that_present_name)) return false; if (!this.name.equals(that.name)) return false; } boolean this_present_num_tasks = true; boolean that_present_num_tasks = true; if (this_present_num_tasks || that_present_num_tasks) { if (!(this_present_num_tasks && that_present_num_tasks)) return false; if (this.num_tasks != that.num_tasks) return false; } boolean this_present_num_executors = true; boolean that_present_num_executors = true; if (this_present_num_executors || that_present_num_executors) { if (!(this_present_num_executors && that_present_num_executors)) return false; if (this.num_executors != that.num_executors) return false; } boolean this_present_num_workers = true; boolean that_present_num_workers = true; if (this_present_num_workers || that_present_num_workers) { if (!(this_present_num_workers && that_present_num_workers)) return false; if (this.num_workers != that.num_workers) return false; } boolean this_present_uptime_secs = true; boolean that_present_uptime_secs = true; if (this_present_uptime_secs || that_present_uptime_secs) { if (!(this_present_uptime_secs && that_present_uptime_secs)) return false; if (this.uptime_secs != that.uptime_secs) return false; } boolean this_present_status = true && this.is_set_status(); boolean that_present_status = true && that.is_set_status(); if (this_present_status || that_present_status) { if (!(this_present_status && that_present_status)) return false; if (!this.status.equals(that.status)) return false; } return true; } @Override public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); boolean present_id = true && (is_set_id()); builder.append(present_id); if (present_id) builder.append(id); boolean present_name = true && (is_set_name()); builder.append(present_name); if (present_name) builder.append(name); boolean present_num_tasks = true; builder.append(present_num_tasks); if (present_num_tasks) builder.append(num_tasks); boolean present_num_executors = true; builder.append(present_num_executors); if (present_num_executors) builder.append(num_executors); boolean present_num_workers = true; builder.append(present_num_workers); if (present_num_workers) builder.append(num_workers); boolean present_uptime_secs = true; builder.append(present_uptime_secs); if (present_uptime_secs) builder.append(uptime_secs); boolean present_status = true && (is_set_status()); builder.append(present_status); if (present_status) builder.append(status); return builder.toHashCode(); } public int compareTo(TopologySummary other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; TopologySummary typedOther = (TopologySummary)other; lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); if (lastComparison != 0) { return lastComparison; } if (is_set_id()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); if (lastComparison != 0) { return lastComparison; } if (is_set_name()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(typedOther.is_set_num_tasks()); if (lastComparison != 0) { return lastComparison; } if (is_set_num_tasks()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_tasks, typedOther.num_tasks); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(typedOther.is_set_num_executors()); if (lastComparison != 0) { return lastComparison; } if (is_set_num_executors()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_executors, typedOther.num_executors); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers()); if (lastComparison != 0) { return lastComparison; } if (is_set_num_workers()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); if (lastComparison != 0) { return lastComparison; } if (is_set_uptime_secs()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); if (lastComparison != 0) { return lastComparison; } } lastComparison = Boolean.valueOf(is_set_status()).compareTo(typedOther.is_set_status()); if (lastComparison != 0) { return lastComparison; } if (is_set_status()) { lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.status, typedOther.status); if (lastComparison != 0) { return lastComparison; } } return 0; } public _Fields fieldForId(int fieldId) { return _Fields.findByThriftId(fieldId); } public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { org.apache.thrift7.protocol.TField field; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == org.apache.thrift7.protocol.TType.STOP) { break; } switch (field.id) { case 1: // ID if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.id = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 2: // NAME if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.name = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 3: // NUM_TASKS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.num_tasks = iprot.readI32(); set_num_tasks_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 4: // NUM_EXECUTORS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.num_executors = iprot.readI32(); set_num_executors_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 5: // NUM_WORKERS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.num_workers = iprot.readI32(); set_num_workers_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 6: // UPTIME_SECS if (field.type == org.apache.thrift7.protocol.TType.I32) { this.uptime_secs = iprot.readI32(); set_uptime_secs_isSet(true); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; case 7: // STATUS if (field.type == org.apache.thrift7.protocol.TType.STRING) { this.status = iprot.readString(); } else { org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } break; default: org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); } iprot.readStructEnd(); validate(); } public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { validate(); oprot.writeStructBegin(STRUCT_DESC); if (this.id != null) { oprot.writeFieldBegin(ID_FIELD_DESC); oprot.writeString(this.id); oprot.writeFieldEnd(); } if (this.name != null) { oprot.writeFieldBegin(NAME_FIELD_DESC); oprot.writeString(this.name); oprot.writeFieldEnd(); } oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC); oprot.writeI32(this.num_tasks); oprot.writeFieldEnd(); oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); oprot.writeI32(this.num_executors); oprot.writeFieldEnd(); oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); oprot.writeI32(this.num_workers); oprot.writeFieldEnd(); oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); oprot.writeI32(this.uptime_secs); oprot.writeFieldEnd(); if (this.status != null) { oprot.writeFieldBegin(STATUS_FIELD_DESC); oprot.writeString(this.status); oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); } @Override public String toString() { StringBuilder sb = new StringBuilder("TopologySummary("); boolean first = true; sb.append("id:"); if (this.id == null) { sb.append("null"); } else { sb.append(this.id); } first = false; if (!first) sb.append(", "); sb.append("name:"); if (this.name == null) { sb.append("null"); } else { sb.append(this.name); } first = false; if (!first) sb.append(", "); sb.append("num_tasks:"); sb.append(this.num_tasks); first = false; if (!first) sb.append(", "); sb.append("num_executors:"); sb.append(this.num_executors); first = false; if (!first) sb.append(", "); sb.append("num_workers:"); sb.append(this.num_workers); first = false; if (!first) sb.append(", "); sb.append("uptime_secs:"); sb.append(this.uptime_secs); first = false; if (!first) sb.append(", "); sb.append("status:"); if (this.status == null) { sb.append("null"); } else { sb.append(this.status); } first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift7.TException { // check for required fields if (!is_set_id()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString()); } if (!is_set_name()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); } if (!is_set_num_tasks()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_tasks' is unset! Struct:" + toString()); } if (!is_set_num_executors()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_executors' is unset! Struct:" + toString()); } if (!is_set_num_workers()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString()); } if (!is_set_uptime_secs()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); } if (!is_set_status()) { throw new org.apache.thrift7.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { try { write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. __isset_bit_vector = new BitSet(1); read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); } catch (org.apache.thrift7.TException te) { throw new java.io.IOException(te); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/grouping/CustomStreamGrouping.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.grouping; import backtype.storm.generated.GlobalStreamId; import backtype.storm.task.WorkerTopologyContext; import java.io.Serializable; import java.util.List; public interface CustomStreamGrouping extends Serializable { /** * Tells the stream grouping at runtime the tasks in the target bolt. * This information should be used in chooseTasks to determine the target tasks. * * It also tells the grouping the metadata on the stream this grouping will be used on. */ void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks); /** * This function implements a custom stream grouping. It takes in as input * the number of tasks in the target bolt in prepare and returns the * tasks to send the tuples to. * * @param values the values to group on */ List chooseTasks(int taskId, List values); } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/BaseTaskHook.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks; import backtype.storm.hooks.info.BoltAckInfo; import backtype.storm.hooks.info.BoltExecuteInfo; import backtype.storm.hooks.info.BoltFailInfo; import backtype.storm.hooks.info.EmitInfo; import backtype.storm.hooks.info.SpoutAckInfo; import backtype.storm.hooks.info.SpoutFailInfo; import backtype.storm.task.TopologyContext; import java.util.Map; public class BaseTaskHook implements ITaskHook { @Override public void prepare(Map conf, TopologyContext context) { } @Override public void cleanup() { } @Override public void emit(EmitInfo info) { } @Override public void spoutAck(SpoutAckInfo info) { } @Override public void spoutFail(SpoutFailInfo info) { } @Override public void boltAck(BoltAckInfo info) { } @Override public void boltFail(BoltFailInfo info) { } @Override public void boltExecute(BoltExecuteInfo info) { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/ITaskHook.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks; import backtype.storm.hooks.info.BoltAckInfo; import backtype.storm.hooks.info.BoltExecuteInfo; import backtype.storm.hooks.info.SpoutFailInfo; import backtype.storm.hooks.info.SpoutAckInfo; import backtype.storm.hooks.info.EmitInfo; import backtype.storm.hooks.info.BoltFailInfo; import backtype.storm.task.TopologyContext; import java.util.Map; public interface ITaskHook { void prepare(Map conf, TopologyContext context); void cleanup(); void emit(EmitInfo info); void spoutAck(SpoutAckInfo info); void spoutFail(SpoutFailInfo info); void boltExecute(BoltExecuteInfo info); void boltAck(BoltAckInfo info); void boltFail(BoltFailInfo info); } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/info/BoltAckInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks.info; import backtype.storm.tuple.Tuple; public class BoltAckInfo { public Tuple tuple; public int ackingTaskId; public Long processLatencyMs; // null if it wasn't sampled public BoltAckInfo(Tuple tuple, int ackingTaskId, Long processLatencyMs) { this.tuple = tuple; this.ackingTaskId = ackingTaskId; this.processLatencyMs = processLatencyMs; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/info/BoltExecuteInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks.info; import backtype.storm.tuple.Tuple; public class BoltExecuteInfo { public Tuple tuple; public int executingTaskId; public Long executeLatencyMs; // null if it wasn't sampled public BoltExecuteInfo(Tuple tuple, int executingTaskId, Long executeLatencyMs) { this.tuple = tuple; this.executingTaskId = executingTaskId; this.executeLatencyMs = executeLatencyMs; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/info/BoltFailInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks.info; import backtype.storm.tuple.Tuple; public class BoltFailInfo { public Tuple tuple; public int failingTaskId; public Long failLatencyMs; // null if it wasn't sampled public BoltFailInfo(Tuple tuple, int failingTaskId, Long failLatencyMs) { this.tuple = tuple; this.failingTaskId = failingTaskId; this.failLatencyMs = failLatencyMs; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/info/EmitInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks.info; import java.util.Collection; import java.util.List; public class EmitInfo { public List values; public String stream; public int taskId; public Collection outTasks; public EmitInfo(List values, String stream, int taskId, Collection outTasks) { this.values = values; this.stream = stream; this.taskId = taskId; this.outTasks = outTasks; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/info/SpoutAckInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks.info; public class SpoutAckInfo { public Object messageId; public int spoutTaskId; public Long completeLatencyMs; // null if it wasn't sampled public SpoutAckInfo(Object messageId, int spoutTaskId, Long completeLatencyMs) { this.messageId = messageId; this.spoutTaskId = spoutTaskId; this.completeLatencyMs = completeLatencyMs; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/hooks/info/SpoutFailInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.hooks.info; public class SpoutFailInfo { public Object messageId; public int spoutTaskId; public Long failLatencyMs; // null if it wasn't sampled public SpoutFailInfo(Object messageId, int spoutTaskId, Long failLatencyMs) { this.messageId = messageId; this.spoutTaskId = spoutTaskId; this.failLatencyMs = failLatencyMs; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/messaging/IConnection.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging; public interface IConnection { /** * receive a message (consists taskId and payload) * @param flags 0: block, 1: non-block * @return */ public TaskMessage recv(int flags); /** * send a message with taskId and payload * @param taskId task ID * @param payload */ public void send(int taskId, byte[] payload); /** * close this connection */ public void close(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/messaging/IContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging; import java.util.Map; /** * This interface needs to be implemented for messaging plugin. * * Messaging plugin is specified via Storm config parameter, storm.messaging.transport. * * A messaging plugin should have a default constructor and implements IContext interface. * Upon construction, we will invoke IContext::prepare(storm_conf) to enable context to be configured * according to storm configuration. */ public interface IContext { /** * This method is invoked at the startup of messaging plugin * @param storm_conf storm configuration */ public void prepare(Map storm_conf); /** * This method is invoked when a worker is unload a messaging plugin */ public void term(); /** * This method establishes a server side connection * @param storm_id topology ID * @param port port # * @return server side connection */ public IConnection bind(String storm_id, int port); /** * This method establish a client side connection to a remote server * @param storm_id topology ID * @param host remote host * @param port remote port * @return client side connection */ public IConnection connect(String storm_id, String host, int port); }; ================================================ FILE: storm-core/src/jvm/backtype/storm/messaging/TaskMessage.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging; import java.nio.ByteBuffer; public class TaskMessage { private int _task; private byte[] _message; public TaskMessage(int task, byte[] message) { _task = task; _message = message; } public int task() { return _task; } public byte[] message() { return _message; } public ByteBuffer serialize() { ByteBuffer bb = ByteBuffer.allocate(_message.length+2); bb.putShort((short)_task); bb.put(_message); return bb; } public void deserialize(ByteBuffer packet) { if (packet==null) return; _task = packet.getShort(); _message = new byte[packet.limit()-2]; packet.get(_message); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/messaging/TransportFactory.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.lang.reflect.Method; import backtype.storm.Config; public class TransportFactory { public static final Logger LOG = LoggerFactory.getLogger(TransportFactory.class); public static IContext makeContext(Map storm_conf) { //get factory class name String transport_plugin_klassName = (String)storm_conf.get(Config.STORM_MESSAGING_TRANSPORT); LOG.info("Storm peer transport plugin:"+transport_plugin_klassName); IContext transport = null; try { //create a factory class Class klass = Class.forName(transport_plugin_klassName); //obtain a context object Object obj = klass.newInstance(); if (obj instanceof IContext) { //case 1: plugin is a IContext class transport = (IContext)obj; //initialize with storm configuration transport.prepare(storm_conf); } else { //case 2: Non-IContext plugin must have a makeContext(storm_conf) method that returns IContext object Method method = klass.getMethod("makeContext", Map.class); LOG.debug("object:"+obj+" method:"+method); transport = (IContext) method.invoke(obj, storm_conf); } } catch(Exception e) { throw new RuntimeException("Fail to construct messaging plugin from plugin "+transport_plugin_klassName, e); } return transport; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/LoggingMetricsConsumer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.Map; import backtype.storm.metric.api.IMetricsConsumer; import backtype.storm.task.IErrorReporter; import backtype.storm.task.TopologyContext; import backtype.storm.utils.Utils; /* * Listens for all metrics, dumps them to log * * To use, add this to your topology's configuration: * conf.registerMetricsConsumer(backtype.storm.metrics.LoggingMetricsConsumer.class, 1); * * Or edit the storm.yaml config file: * * topology.metrics.consumer.register: * - class: "backtype.storm.metrics.LoggingMetricsConsumer" * parallelism.hint: 1 * */ public class LoggingMetricsConsumer implements IMetricsConsumer { public static final Logger LOG = LoggerFactory.getLogger(LoggingMetricsConsumer.class); @Override public void prepare(Map stormConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { } static private String padding = " "; @Override public void handleDataPoints(TaskInfo taskInfo, Collection dataPoints) { StringBuilder sb = new StringBuilder(); String header = String.format("%d\t%15s:%-4d\t%3d:%-11s\t", taskInfo.timestamp, taskInfo.srcWorkerHost, taskInfo.srcWorkerPort, taskInfo.srcTaskId, taskInfo.srcComponentId); sb.append(header); for (DataPoint p : dataPoints) { sb.delete(header.length(), sb.length()); sb.append(p.name) .append(padding).delete(header.length()+23,sb.length()).append("\t") .append(p.value); LOG.info(sb.toString()); } } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/MetricsConsumerBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric; import backtype.storm.Config; import backtype.storm.metric.api.IMetricsConsumer; import backtype.storm.task.IBolt; import backtype.storm.task.IErrorReporter; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Tuple; import java.util.Collection; import java.util.Map; public class MetricsConsumerBolt implements IBolt { IMetricsConsumer _metricsConsumer; String _consumerClassName; OutputCollector _collector; Object _registrationArgument; public MetricsConsumerBolt(String consumerClassName, Object registrationArgument) { _consumerClassName = consumerClassName; _registrationArgument = registrationArgument; } @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { try { _metricsConsumer = (IMetricsConsumer)Class.forName(_consumerClassName).newInstance(); } catch (Exception e) { throw new RuntimeException("Could not instantiate a class listed in config under section " + Config.TOPOLOGY_METRICS_CONSUMER_REGISTER + " with fully qualified name " + _consumerClassName, e); } _metricsConsumer.prepare(stormConf, _registrationArgument, context, (IErrorReporter)collector); _collector = collector; } @Override public void execute(Tuple input) { _metricsConsumer.handleDataPoints((IMetricsConsumer.TaskInfo)input.getValue(0), (Collection)input.getValue(1)); _collector.ack(input); } @Override public void cleanup() { _metricsConsumer.cleanup(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/SystemBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric; import backtype.storm.Config; import backtype.storm.metric.api.AssignableMetric; import backtype.storm.metric.api.IMetric; import backtype.storm.task.IBolt; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Tuple; import clojure.lang.AFn; import clojure.lang.IFn; import clojure.lang.RT; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.lang.management.*; import java.util.HashMap; import java.util.List; import java.util.Map; // There is one task inside one executor for each worker of the topology. // TaskID is always -1, therefore you can only send-unanchored tuples to co-located SystemBolt. // This bolt was conceived to export worker stats via metrics api. public class SystemBolt implements IBolt { private static Logger LOG = LoggerFactory.getLogger(SystemBolt.class); private static boolean _prepareWasCalled = false; private static class MemoryUsageMetric implements IMetric { IFn _getUsage; public MemoryUsageMetric(IFn getUsage) { _getUsage = getUsage; } @Override public Object getValueAndReset() { MemoryUsage memUsage = (MemoryUsage)_getUsage.invoke(); HashMap m = new HashMap(); m.put("maxBytes", memUsage.getMax()); m.put("committedBytes", memUsage.getCommitted()); m.put("initBytes", memUsage.getInit()); m.put("usedBytes", memUsage.getUsed()); m.put("virtualFreeBytes", memUsage.getMax() - memUsage.getUsed()); m.put("unusedBytes", memUsage.getCommitted() - memUsage.getUsed()); return m; } } // canonically the metrics data exported is time bucketed when doing counts. // convert the absolute values here into time buckets. private static class GarbageCollectorMetric implements IMetric { GarbageCollectorMXBean _gcBean; Long _collectionCount; Long _collectionTime; public GarbageCollectorMetric(GarbageCollectorMXBean gcBean) { _gcBean = gcBean; } @Override public Object getValueAndReset() { Long collectionCountP = _gcBean.getCollectionCount(); Long collectionTimeP = _gcBean.getCollectionTime(); Map ret = null; if(_collectionCount!=null && _collectionTime!=null) { ret = new HashMap(); ret.put("count", collectionCountP - _collectionCount); ret.put("timeMs", collectionTimeP - _collectionTime); } _collectionCount = collectionCountP; _collectionTime = collectionTimeP; return ret; } } @Override public void prepare(final Map stormConf, TopologyContext context, OutputCollector collector) { if(_prepareWasCalled && !"local".equals(stormConf.get(Config.STORM_CLUSTER_MODE))) { throw new RuntimeException("A single worker should have 1 SystemBolt instance."); } _prepareWasCalled = true; int bucketSize = RT.intCast(stormConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)); final RuntimeMXBean jvmRT = ManagementFactory.getRuntimeMXBean(); context.registerMetric("uptimeSecs", new IMetric() { @Override public Object getValueAndReset() { return jvmRT.getUptime()/1000.0; } }, bucketSize); context.registerMetric("startTimeSecs", new IMetric() { @Override public Object getValueAndReset() { return jvmRT.getStartTime()/1000.0; } }, bucketSize); context.registerMetric("newWorkerEvent", new IMetric() { boolean doEvent = true; @Override public Object getValueAndReset() { if (doEvent) { doEvent = false; return 1; } else return 0; } }, bucketSize); final MemoryMXBean jvmMemRT = ManagementFactory.getMemoryMXBean(); context.registerMetric("memory/heap", new MemoryUsageMetric(new AFn() { public Object invoke() { return jvmMemRT.getHeapMemoryUsage(); } }), bucketSize); context.registerMetric("memory/nonHeap", new MemoryUsageMetric(new AFn() { public Object invoke() { return jvmMemRT.getNonHeapMemoryUsage(); } }), bucketSize); for(GarbageCollectorMXBean b : ManagementFactory.getGarbageCollectorMXBeans()) { context.registerMetric("GC/" + b.getName().replaceAll("\\W", ""), new GarbageCollectorMetric(b), bucketSize); } } @Override public void execute(Tuple input) { throw new RuntimeException("Non-system tuples should never be sent to __system bolt."); } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/AssignableMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public class AssignableMetric implements IMetric { Object _value; public AssignableMetric(Object value) { _value = value; } public void setValue(Object value) { _value = value; } public Object getValueAndReset() { return _value; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/CombinedMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public class CombinedMetric implements IMetric { private final ICombiner _combiner; private Object _value; public CombinedMetric(ICombiner combiner) { _combiner = combiner; _value = _combiner.identity(); } public void update(Object value) { _value = _combiner.combine(_value, value); } public Object getValueAndReset() { Object ret = _value; _value = _combiner.identity(); return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; import backtype.storm.metric.api.IMetric; public class CountMetric implements IMetric { long _value = 0; public CountMetric() { } public void incr() { _value++; } public void incrBy(long incrementBy) { _value += incrementBy; } public Object getValueAndReset() { long ret = _value; _value = 0; return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/ICombiner.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public interface ICombiner { public T identity(); public T combine(T a, T b); } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/IMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public interface IMetric { public Object getValueAndReset(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/IMetricsConsumer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; import backtype.storm.task.IErrorReporter; import backtype.storm.task.TopologyContext; import java.util.Collection; import java.util.Map; public interface IMetricsConsumer { public static class TaskInfo { public TaskInfo() {} public TaskInfo(String srcWorkerHost, int srcWorkerPort, String srcComponentId, int srcTaskId, long timestamp, int updateIntervalSecs) { this.srcWorkerHost = srcWorkerHost; this.srcWorkerPort = srcWorkerPort; this.srcComponentId = srcComponentId; this.srcTaskId = srcTaskId; this.timestamp = timestamp; this.updateIntervalSecs = updateIntervalSecs; } public String srcWorkerHost; public int srcWorkerPort; public String srcComponentId; public int srcTaskId; public long timestamp; public int updateIntervalSecs; } public static class DataPoint { public DataPoint() {} public DataPoint(String name, Object value) { this.name = name; this.value = value; } @Override public String toString() { return "[" + name + " = " + value + "]"; } public String name; public Object value; } void prepare(Map stormConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter); void handleDataPoints(TaskInfo taskInfo, Collection dataPoints); void cleanup(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/IReducer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public interface IReducer { T init(); T reduce(T accumulator, Object input); Object extractResult(T accumulator); } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/IStatefulObject.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public interface IStatefulObject { Object getState(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; import backtype.storm.metric.api.IReducer; class MeanReducerState { public int count = 0; public double sum = 0.0; } public class MeanReducer implements IReducer { public MeanReducerState init() { return new MeanReducerState(); } public MeanReducerState reduce(MeanReducerState acc, Object input) { acc.count++; if(input instanceof Double) { acc.sum += (Double)input; } else if(input instanceof Long) { acc.sum += ((Long)input).doubleValue(); } else if(input instanceof Integer) { acc.sum += ((Integer)input).doubleValue(); } else { throw new RuntimeException( "MeanReducer::reduce called with unsupported input type `" + input.getClass() + "`. Supported types are Double, Long, Integer."); } return acc; } public Object extractResult(MeanReducerState acc) { if(acc.count > 0) { return new Double(acc.sum / (double)acc.count); } else { return null; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; import backtype.storm.metric.api.IMetric; import java.util.HashMap; import java.util.Map; public class MultiCountMetric implements IMetric { Map _value = new HashMap(); public MultiCountMetric() { } public CountMetric scope(String key) { CountMetric val = _value.get(key); if(val == null) { _value.put(key, val = new CountMetric()); } return val; } public Object getValueAndReset() { Map ret = new HashMap(); for(Map.Entry e : _value.entrySet()) { ret.put(e.getKey(), e.getValue().getValueAndReset()); } return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; import backtype.storm.metric.api.IMetric; import java.util.HashMap; import java.util.Map; public class MultiReducedMetric implements IMetric { Map _value = new HashMap(); IReducer _reducer; public MultiReducedMetric(IReducer reducer) { _reducer = reducer; } public ReducedMetric scope(String key) { ReducedMetric val = _value.get(key); if(val == null) { _value.put(key, val = new ReducedMetric(_reducer)); } return val; } public Object getValueAndReset() { Map ret = new HashMap(); for(Map.Entry e : _value.entrySet()) { Object val = e.getValue().getValueAndReset(); if(val != null) { ret.put(e.getKey(), val); } } return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/ReducedMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public class ReducedMetric implements IMetric { private final IReducer _reducer; private Object _accumulator; public ReducedMetric(IReducer reducer) { _reducer = reducer; _accumulator = _reducer.init(); } public void update(Object value) { _accumulator = _reducer.reduce(_accumulator, value); } public Object getValueAndReset() { Object ret = _reducer.extractResult(_accumulator); _accumulator = _reducer.init(); return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/metric/api/StateMetric.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.metric.api; public class StateMetric implements IMetric { private IStatefulObject _obj; public StateMetric(IStatefulObject obj) { _obj = obj; } @Override public Object getValueAndReset() { return _obj.getState(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/nimbus/DefaultTopologyValidator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.nimbus; import backtype.storm.generated.InvalidTopologyException; import backtype.storm.generated.StormTopology; import java.util.Map; public class DefaultTopologyValidator implements ITopologyValidator { @Override public void prepare(Map StormConf){ } @Override public void validate(String topologyName, Map topologyConf, StormTopology topology) throws InvalidTopologyException { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/nimbus/ITopologyValidator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.nimbus; import backtype.storm.generated.InvalidTopologyException; import backtype.storm.generated.StormTopology; import java.util.Map; public interface ITopologyValidator { void prepare(Map StormConf); void validate(String topologyName, Map topologyConf, StormTopology topology) throws InvalidTopologyException; } ================================================ FILE: storm-core/src/jvm/backtype/storm/planner/CompoundSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.planner; public class CompoundSpout //implements ISpout { } ================================================ FILE: storm-core/src/jvm/backtype/storm/planner/CompoundTask.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.planner; public class CompoundTask // implements IBolt { } ================================================ FILE: storm-core/src/jvm/backtype/storm/planner/TaskBundle.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.planner; import backtype.storm.task.IBolt; import java.io.Serializable; public class TaskBundle implements Serializable { public IBolt task; public int componentId; public TaskBundle(IBolt task, int componentId) { this.task = task; this.componentId = componentId; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/Cluster.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; public class Cluster { /** * key: supervisor id, value: supervisor details */ private Map supervisors; /** * key: topologyId, value: topology's current assignments. */ private Map assignments; /** * a map from hostname to supervisor id. */ private Map> hostToId; private Set blackListedHosts = new HashSet(); private INimbus inimbus; public Cluster(INimbus nimbus, Map supervisors, Map assignments){ this.inimbus = nimbus; this.supervisors = new HashMap(supervisors.size()); this.supervisors.putAll(supervisors); this.assignments = new HashMap(assignments.size()); this.assignments.putAll(assignments); this.hostToId = new HashMap>(); for (String nodeId : supervisors.keySet()) { SupervisorDetails supervisor = supervisors.get(nodeId); String host = supervisor.getHost(); if (!this.hostToId.containsKey(host)) { this.hostToId.put(host, new ArrayList()); } this.hostToId.get(host).add(nodeId); } } public void setBlacklistedHosts(Set hosts) { blackListedHosts = hosts; } public Set getBlacklistedHosts() { return blackListedHosts; } public void blacklistHost(String host) { // this is so it plays well with setting blackListedHosts to an immutable list if(blackListedHosts==null) blackListedHosts = new HashSet(); if(!(blackListedHosts instanceof HashSet)) blackListedHosts = new HashSet(blackListedHosts); blackListedHosts.add(host); } public boolean isBlackListed(String supervisorId) { return blackListedHosts != null && blackListedHosts.contains(getHost(supervisorId)); } public boolean isBlacklistedHost(String host) { return blackListedHosts != null && blackListedHosts.contains(host); } public String getHost(String supervisorId) { return inimbus.getHostName(supervisors, supervisorId); } /** * Gets all the topologies which needs scheduling. * * @param topologies * @return */ public List needsSchedulingTopologies(Topologies topologies) { List ret = new ArrayList(); for (TopologyDetails topology : topologies.getTopologies()) { if (needsScheduling(topology)) { ret.add(topology); } } return ret; } /** * Does the topology need scheduling? * * A topology needs scheduling if one of the following conditions holds: *
    *
  • Although the topology is assigned slots, but is squeezed. i.e. the topology is assigned less slots than desired.
  • *
  • There are unassigned executors in this topology
  • *
*/ public boolean needsScheduling(TopologyDetails topology) { int desiredNumWorkers = topology.getNumWorkers(); int assignedNumWorkers = this.getAssignedNumWorkers(topology); if (desiredNumWorkers > assignedNumWorkers) { return true; } return this.getUnassignedExecutors(topology).size() > 0; } /** * Gets a executor -> component-id map which needs scheduling in this topology. * * @param topology * @return */ public Map getNeedsSchedulingExecutorToComponents(TopologyDetails topology) { Collection allExecutors = new HashSet(topology.getExecutors()); SchedulerAssignment assignment = this.assignments.get(topology.getId()); if (assignment != null) { Collection assignedExecutors = assignment.getExecutors(); allExecutors.removeAll(assignedExecutors); } return topology.selectExecutorToComponent(allExecutors); } /** * Gets a component-id -> executors map which needs scheduling in this topology. * * @param topology * @return */ public Map> getNeedsSchedulingComponentToExecutors(TopologyDetails topology) { Map executorToComponents = this.getNeedsSchedulingExecutorToComponents(topology); Map> componentToExecutors = new HashMap>(); for (ExecutorDetails executor : executorToComponents.keySet()) { String component = executorToComponents.get(executor); if (!componentToExecutors.containsKey(component)) { componentToExecutors.put(component, new ArrayList()); } componentToExecutors.get(component).add(executor); } return componentToExecutors; } /** * Get all the used ports of this supervisor. * * @param cluster * @return */ public Set getUsedPorts(SupervisorDetails supervisor) { Map assignments = this.getAssignments(); Set usedPorts = new HashSet(); for (SchedulerAssignment assignment : assignments.values()) { for (WorkerSlot slot : assignment.getExecutorToSlot().values()) { if (slot.getNodeId().equals(supervisor.getId())) { usedPorts.add(slot.getPort()); } } } return usedPorts; } /** * Return the available ports of this supervisor. * * @param cluster * @return */ public Set getAvailablePorts(SupervisorDetails supervisor) { Set usedPorts = this.getUsedPorts(supervisor); Set ret = new HashSet(); ret.addAll(getAssignablePorts(supervisor)); ret.removeAll(usedPorts); return ret; } public Set getAssignablePorts(SupervisorDetails supervisor) { if(isBlackListed(supervisor.id)) return new HashSet(); return supervisor.allPorts; } /** * Return all the available slots on this supervisor. * * @param cluster * @return */ public List getAvailableSlots(SupervisorDetails supervisor) { Set ports = this.getAvailablePorts(supervisor); List slots = new ArrayList(ports.size()); for (Integer port : ports) { slots.add(new WorkerSlot(supervisor.getId(), port)); } return slots; } public List getAssignableSlots(SupervisorDetails supervisor) { Set ports = this.getAssignablePorts(supervisor); List slots = new ArrayList(ports.size()); for (Integer port : ports) { slots.add(new WorkerSlot(supervisor.getId(), port)); } return slots; } /** * get the unassigned executors of the topology. */ public Collection getUnassignedExecutors(TopologyDetails topology) { if (topology == null) { return new ArrayList(0); } Collection ret = new HashSet(topology.getExecutors()); SchedulerAssignment assignment = this.getAssignmentById(topology.getId()); if (assignment != null) { Set assignedExecutors = assignment.getExecutors(); ret.removeAll(assignedExecutors); } return ret; } /** * Gets the number of workers assigned to this topology. * * @param topology * @return */ public int getAssignedNumWorkers(TopologyDetails topology) { SchedulerAssignment assignment = this.getAssignmentById(topology.getId()); if (topology == null || assignment == null) { return 0; } Set slots = new HashSet(); slots.addAll(assignment.getExecutorToSlot().values()); return slots.size(); } /** * Assign the slot to the executors for this topology. * * @throws RuntimeException if the specified slot is already occupied. */ public void assign(WorkerSlot slot, String topologyId, Collection executors) { if (this.isSlotOccupied(slot)) { throw new RuntimeException("slot: [" + slot.getNodeId() + ", " + slot.getPort() + "] is already occupied."); } SchedulerAssignmentImpl assignment = (SchedulerAssignmentImpl)this.getAssignmentById(topologyId); if (assignment == null) { assignment = new SchedulerAssignmentImpl(topologyId, new HashMap()); this.assignments.put(topologyId, assignment); } else { for (ExecutorDetails executor : executors) { if (assignment.isExecutorAssigned(executor)) { throw new RuntimeException("the executor is already assigned, you should unassign it before assign it to another slot."); } } } assignment.assign(slot, executors); } /** * Gets all the available slots in the cluster. * * @return */ public List getAvailableSlots() { List slots = new ArrayList(); for (SupervisorDetails supervisor : this.supervisors.values()) { slots.addAll(this.getAvailableSlots(supervisor)); } return slots; } public List getAssignableSlots() { List slots = new ArrayList(); for (SupervisorDetails supervisor : this.supervisors.values()) { slots.addAll(this.getAssignableSlots(supervisor)); } return slots; } /** * Free the specified slot. * * @param slot */ public void freeSlot(WorkerSlot slot) { // remove the slot from the existing assignments for (SchedulerAssignmentImpl assignment : this.assignments.values()) { if (assignment.isSlotOccupied(slot)) { assignment.unassignBySlot(slot); } } } /** * free the slots. * * @param slots */ public void freeSlots(Collection slots) { if(slots!=null) { for (WorkerSlot slot : slots) { this.freeSlot(slot); } } } /** * Checks the specified slot is occupied. * * @param slot the slot be to checked. * @return */ public boolean isSlotOccupied(WorkerSlot slot) { for (SchedulerAssignment assignment : this.assignments.values()) { if (assignment.isSlotOccupied(slot)) { return true; } } return false; } /** * get the current assignment for the topology. */ public SchedulerAssignment getAssignmentById(String topologyId) { if (this.assignments.containsKey(topologyId)) { return this.assignments.get(topologyId); } return null; } /** * Get a specific supervisor with the nodeId */ public SupervisorDetails getSupervisorById(String nodeId) { if (this.supervisors.containsKey(nodeId)) { return this.supervisors.get(nodeId); } return null; } public Collection getUsedSlots() { Set ret = new HashSet(); for(SchedulerAssignmentImpl s: assignments.values()) { ret.addAll(s.getExecutorToSlot().values()); } return ret; } /** * Get all the supervisors on the specified host. * * @param host hostname of the supervisor * @return the SupervisorDetails object. */ public List getSupervisorsByHost(String host) { List nodeIds = this.hostToId.get(host); List ret = new ArrayList(); if (nodeIds != null) { for (String nodeId : nodeIds) { ret.add(this.getSupervisorById(nodeId)); } } return ret; } /** * Get all the assignments. */ public Map getAssignments() { Map ret = new HashMap(this.assignments.size()); for (String topologyId : this.assignments.keySet()) { ret.put(topologyId, this.assignments.get(topologyId)); } return ret; } /** * Get all the supervisors. */ public Map getSupervisors() { return this.supervisors; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/ExecutorDetails.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; public class ExecutorDetails { int startTask; int endTask; public ExecutorDetails(int startTask, int endTask){ this.startTask = startTask; this.endTask = endTask; } public int getStartTask() { return startTask; } public int getEndTask() { return endTask; } public boolean equals(Object other) { if (other == null || !(other instanceof ExecutorDetails)) { return false; } ExecutorDetails executor = (ExecutorDetails)other; return (this.startTask == executor.startTask) && (this.endTask == executor.endTask); } public int hashCode() { return this.startTask + 13 * this.endTask; } @Override public String toString() { return "[" + this.startTask + ", " + this.endTask + "]"; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/INimbus.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.Collection; import java.util.Map; import java.util.Set; public interface INimbus { void prepare(Map stormConf, String schedulerLocalDir); /** * Returns all slots that are available for the next round of scheduling. A slot is available for scheduling * if it is free and can be assigned to, or if it is used and can be reassigned. */ Collection allSlotsAvailableForScheduling(Collection existingSupervisors, Topologies topologies, Set topologiesMissingAssignments); // this is called after the assignment is changed in ZK void assignSlots(Topologies topologies, Map> newSlotsByTopologyId); // map from node id to supervisor details String getHostName(Map existingSupervisors, String nodeId); IScheduler getForcedScheduler(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/IScheduler.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.Map; public interface IScheduler { void prepare(Map conf); /** * Set assignments for the topologies which needs scheduling. The new assignments is available * through cluster.getAssignments() * *@param topologies all the topologies in the cluster, some of them need schedule. Topologies object here * only contain static information about topologies. Information like assignments, slots are all in * the clusterobject. *@param cluster the cluster these topologies are running in. cluster contains everything user * need to develop a new scheduling logic. e.g. supervisors information, available slots, current * assignments for all the topologies etc. User can set the new assignment for topologies using * cluster.setAssignmentById */ void schedule(Topologies topologies, Cluster cluster); } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/ISupervisor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.Map; import java.util.Collection; public interface ISupervisor { void prepare(Map stormConf, String schedulerLocalDir); // for mesos, this is {hostname}-{topologyid} /** * The id used for writing metadata into ZK. */ String getSupervisorId(); /** * The id used in assignments. This combined with confirmAssigned decides what * this supervisor is responsible for. The combination of this and getSupervisorId * allows Nimbus to assign to a single machine and have multiple supervisors * on that machine execute the assignment. This is important for achieving resource isolation. */ String getAssignmentId(); Object getMetadata(); boolean confirmAssigned(int port); // calls this before actually killing the worker locally... // sends a "task finished" update void killedWorker(int port); void assigned(Collection ports); } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignment.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.Map; import java.util.Set; public interface SchedulerAssignment { /** * Does this slot occupied by this assignment? * @param slot * @return */ public boolean isSlotOccupied(WorkerSlot slot); /** * is the executor assigned? * * @param executor * @return */ public boolean isExecutorAssigned(ExecutorDetails executor); /** * get the topology-id this assignment is for. * @return */ public String getTopologyId(); /** * get the executor -> slot map. * @return */ public Map getExecutorToSlot(); /** * Return the executors covered by this assignments * @return */ public Set getExecutors(); public Set getSlots(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignmentImpl.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; //TODO: improve this by maintaining slot -> executors as well for more efficient operations public class SchedulerAssignmentImpl implements SchedulerAssignment { /** * topology-id this assignment is for. */ String topologyId; /** * assignment detail, a mapping from executor to WorkerSlot */ Map executorToSlot; public SchedulerAssignmentImpl(String topologyId, Map executorToSlots) { this.topologyId = topologyId; this.executorToSlot = new HashMap(0); if (executorToSlots != null) { this.executorToSlot.putAll(executorToSlots); } } @Override public Set getSlots() { return new HashSet(executorToSlot.values()); } /** * Assign the slot to executors. * @param slot * @param executors */ public void assign(WorkerSlot slot, Collection executors) { for (ExecutorDetails executor : executors) { this.executorToSlot.put(executor, slot); } } /** * Release the slot occupied by this assignment. * @param slot */ public void unassignBySlot(WorkerSlot slot) { List executors = new ArrayList(); for (ExecutorDetails executor : this.executorToSlot.keySet()) { WorkerSlot ws = this.executorToSlot.get(executor); if (ws.equals(slot)) { executors.add(executor); } } // remove for (ExecutorDetails executor : executors) { this.executorToSlot.remove(executor); } } /** * Does this slot occupied by this assignment? * @param slot * @return */ public boolean isSlotOccupied(WorkerSlot slot) { return this.executorToSlot.containsValue(slot); } public boolean isExecutorAssigned(ExecutorDetails executor) { return this.executorToSlot.containsKey(executor); } public String getTopologyId() { return this.topologyId; } public Map getExecutorToSlot() { return this.executorToSlot; } /** * Return the executors covered by this assignments * @return */ public Set getExecutors() { return this.executorToSlot.keySet(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.Collection; import java.util.HashSet; import java.util.Set; public class SupervisorDetails { String id; /** * hostname of this supervisor */ String host; Object meta; /** * meta data configured for this supervisor */ Object schedulerMeta; /** * all the ports of the supervisor */ Set allPorts; public SupervisorDetails(String id, Object meta){ this.id = id; this.meta = meta; allPorts = new HashSet(); } public SupervisorDetails(String id, Object meta, Collection allPorts){ this.id = id; this.meta = meta; setAllPorts(allPorts); } public SupervisorDetails(String id, String host, Object schedulerMeta, Collection allPorts){ this.id = id; this.host = host; this.schedulerMeta = schedulerMeta; setAllPorts(allPorts); } private void setAllPorts(Collection allPorts) { this.allPorts = new HashSet(); if(allPorts!=null) { for(Number n: allPorts) { this.allPorts.add(n.intValue()); } } } public String getId() { return id; } public String getHost() { return host; } public Object getMeta() { return meta; } public Set getAllPorts() { return allPorts; } public Object getSchedulerMeta() { return this.schedulerMeta; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/Topologies.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.Collection; import java.util.HashMap; import java.util.Map; public class Topologies { Map topologies; Map nameToId; public Topologies(Map topologies) { if(topologies==null) topologies = new HashMap(); this.topologies = new HashMap(topologies.size()); this.topologies.putAll(topologies); this.nameToId = new HashMap(topologies.size()); for (String topologyId : topologies.keySet()) { TopologyDetails topology = topologies.get(topologyId); this.nameToId.put(topology.getName(), topologyId); } } public TopologyDetails getById(String topologyId) { return this.topologies.get(topologyId); } public TopologyDetails getByName(String topologyName) { String topologyId = this.nameToId.get(topologyName); if (topologyId == null) { return null; } else { return this.getById(topologyId); } } public Collection getTopologies() { return this.topologies.values(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; import java.util.Collection; import java.util.HashMap; import java.util.Map; import backtype.storm.Config; import backtype.storm.generated.StormTopology; public class TopologyDetails { String topologyId; Map topologyConf; StormTopology topology; Map executorToComponent; int numWorkers; public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers) { this.topologyId = topologyId; this.topologyConf = topologyConf; this.topology = topology; this.numWorkers = numWorkers; } public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers, Map executorToComponents) { this(topologyId, topologyConf, topology, numWorkers); this.executorToComponent = new HashMap(0); if (executorToComponents != null) { this.executorToComponent.putAll(executorToComponents); } } public String getId() { return topologyId; } public String getName() { return (String)this.topologyConf.get(Config.TOPOLOGY_NAME); } public Map getConf() { return topologyConf; } public int getNumWorkers() { return numWorkers; } public StormTopology getTopology() { return topology; } public Map getExecutorToComponent() { return this.executorToComponent; } public Map selectExecutorToComponent(Collection executors) { Map ret = new HashMap(executors.size()); for (ExecutorDetails executor : executors) { String compId = this.executorToComponent.get(executor); if (compId != null) { ret.put(executor, compId); } } return ret; } public Collection getExecutors() { return this.executorToComponent.keySet(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.scheduler; public class WorkerSlot { String nodeId; int port; public WorkerSlot(String nodeId, Number port) { this.nodeId = nodeId; this.port = port.intValue(); } public String getNodeId() { return nodeId; } public int getPort() { return port; } @Override public int hashCode() { return nodeId.hashCode() + 13 * ((Integer) port).hashCode(); } @Override public boolean equals(Object o) { WorkerSlot other = (WorkerSlot) o; return this.port == other.port && this.nodeId.equals(other.nodeId); } @Override public String toString() { return this.nodeId + ":" + this.port; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import backtype.storm.Config; import javax.security.auth.login.Configuration; import javax.security.auth.login.AppConfigurationEntry; import java.security.NoSuchAlgorithmException; import java.security.URIParameter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; import java.util.Map; public class AuthUtils { private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class); public static final String LOGIN_CONTEXT_SERVER = "StormServer"; public static final String LOGIN_CONTEXT_CLIENT = "StormClient"; public static final String SERVICE = "storm_thrift_server"; /** * Construct a JAAS configuration object per storm configuration file * @param storm_conf Storm configuration * @return JAAS configuration object */ public static Configuration GetConfiguration(Map storm_conf) { Configuration login_conf = null; //find login file configuration from Storm configuration String loginConfigurationFile = (String)storm_conf.get("java.security.auth.login.config"); if ((loginConfigurationFile != null) && (loginConfigurationFile.length()>0)) { try { URI config_uri = new File(loginConfigurationFile).toURI(); login_conf = Configuration.getInstance("JavaLoginConfig", new URIParameter(config_uri)); } catch (NoSuchAlgorithmException ex1) { if (ex1.getCause() instanceof FileNotFoundException) throw new RuntimeException("configuration file "+loginConfigurationFile+" could not be found"); else throw new RuntimeException(ex1); } catch (Exception ex2) { throw new RuntimeException(ex2); } } return login_conf; } /** * Construct a transport plugin per storm configuration * @param conf storm configuration * @return */ public static ITransportPlugin GetTransportPlugin(Map storm_conf, Configuration login_conf) { ITransportPlugin transportPlugin = null; try { String transport_plugin_klassName = (String) storm_conf.get(Config.STORM_THRIFT_TRANSPORT_PLUGIN); Class klass = Class.forName(transport_plugin_klassName); transportPlugin = (ITransportPlugin)klass.newInstance(); transportPlugin.prepare(storm_conf, login_conf); } catch(Exception e) { throw new RuntimeException(e); } return transportPlugin; } public static String get(Configuration configuration, String section, String key) throws IOException { AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(section); if (configurationEntries == null) { String errorMessage = "Could not find a '"+ section + "' entry in this configuration."; throw new IOException(errorMessage); } for(AppConfigurationEntry entry: configurationEntries) { Object val = entry.getOptions().get(key); if (val != null) return (String)val; } return null; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/IAuthorizer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import java.util.Map; /** * Nimbus could be configured with an authorization plugin. * If not specified, all requests are authorized. * * You could specify the authorization plugin via storm parameter. For example: * storm -c nimbus.authorization.class=backtype.storm.security.auth.NoopAuthorizer ... * * You could also specify it via storm.yaml: * nimbus.authorization.class: backtype.storm.security.auth.NoopAuthorizer */ public interface IAuthorizer { /** * Invoked once immediately after construction * @param conf Storm configuration */ void prepare(Map storm_conf); /** * permit() method is invoked for each incoming Thrift request. * @param context request context includes info about * @param operation operation name * @param topology_storm configuration of targeted topology * @return true if the request is authorized, false if reject */ public boolean permit(ReqContext context, String operation, Map topology_conf); } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import java.io.IOException; import java.util.Map; import javax.security.auth.login.Configuration; import org.apache.thrift7.TProcessor; import org.apache.thrift7.server.TServer; import org.apache.thrift7.transport.TTransport; import org.apache.thrift7.transport.TTransportException; /** * Interface for Thrift Transport plugin */ public interface ITransportPlugin { /** * Invoked once immediately after construction * @param storm_conf Storm configuration * @param login_conf login configuration */ void prepare(Map storm_conf, Configuration login_conf); /** * Create a server associated with a given port and service handler * @param port listening port * @param processor service handler * @return server to be binded */ public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException; /** * Connect to the specified server via framed transport * @param transport The underlying Thrift transport. * @param serverHost server host */ public TTransport connect(TTransport transport, String serverHost) throws IOException, TTransportException; } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.net.InetAddress; import com.google.common.annotations.VisibleForTesting; import java.security.AccessControlContext; import java.security.AccessController; import java.security.Principal; import javax.security.auth.Subject; /** * context request context includes info about * (1) remote address, * (2) remote subject and primary principal * (3) request ID */ public class ReqContext { private static final AtomicInteger uniqueId = new AtomicInteger(0); private Subject _subject; private InetAddress _remoteAddr; private Integer _reqID; private Map _storm_conf; /** * Get a request context associated with current thread * @return */ public static ReqContext context() { return ctxt.get(); } //each thread will have its own request context private static final ThreadLocal < ReqContext > ctxt = new ThreadLocal < ReqContext > () { @Override protected ReqContext initialValue() { return new ReqContext(AccessController.getContext()); } }; //private constructor @VisibleForTesting ReqContext(AccessControlContext acl_ctxt) { _subject = Subject.getSubject(acl_ctxt); _reqID = uniqueId.incrementAndGet(); } /** * client address */ public void setRemoteAddress(InetAddress addr) { _remoteAddr = addr; } public InetAddress remoteAddress() { return _remoteAddr; } /** * Set remote subject explicitly */ public void setSubject(Subject subject) { _subject = subject; } /** * Retrieve client subject associated with this request context */ public Subject subject() { return _subject; } /** * The primary principal associated current subject */ public Principal principal() { if (_subject == null) return null; Set princs = _subject.getPrincipals(); if (princs.size()==0) return null; return (Principal) (princs.toArray()[0]); } /** * request ID of this request */ public Integer requestID() { return _reqID; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import java.io.IOException; import java.net.Socket; import java.security.Principal; import java.util.Map; import javax.security.auth.Subject; import javax.security.auth.login.Configuration; import javax.security.sasl.SaslServer; import org.apache.thrift7.TException; import org.apache.thrift7.TProcessor; import org.apache.thrift7.protocol.TBinaryProtocol; import org.apache.thrift7.protocol.TProtocol; import org.apache.thrift7.server.TServer; import org.apache.thrift7.server.TThreadPoolServer; import org.apache.thrift7.transport.TSaslServerTransport; import org.apache.thrift7.transport.TServerSocket; import org.apache.thrift7.transport.TSocket; import org.apache.thrift7.transport.TTransport; import org.apache.thrift7.transport.TTransportException; import org.apache.thrift7.transport.TTransportFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Base class for SASL authentication plugin. */ public abstract class SaslTransportPlugin implements ITransportPlugin { protected Configuration login_conf; private static final Logger LOG = LoggerFactory.getLogger(SaslTransportPlugin.class); /** * Invoked once immediately after construction * @param conf Storm configuration * @param login_conf login configuration */ public void prepare(Map storm_conf, Configuration login_conf) { this.login_conf = login_conf; } public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException { TTransportFactory serverTransportFactory = getServerTransportFactory(); //define THsHaServer args //original: THsHaServer + TNonblockingServerSocket //option: TThreadPoolServer + TServerSocket TServerSocket serverTransport = new TServerSocket(port); TThreadPoolServer.Args server_args = new TThreadPoolServer.Args(serverTransport). processor(new TUGIWrapProcessor(processor)). minWorkerThreads(64). maxWorkerThreads(64). protocolFactory(new TBinaryProtocol.Factory()); if (serverTransportFactory != null) server_args.transportFactory(serverTransportFactory); //construct THsHaServer return new TThreadPoolServer(server_args); } /** * All subclass must implement this method * @return * @throws IOException */ protected abstract TTransportFactory getServerTransportFactory() throws IOException; /** * Processor that pulls the SaslServer object out of the transport, and * assumes the remote user's UGI before calling through to the original * processor. * * This is used on the server side to set the UGI for each specific call. */ private class TUGIWrapProcessor implements TProcessor { final TProcessor wrapped; TUGIWrapProcessor(TProcessor wrapped) { this.wrapped = wrapped; } public boolean process(final TProtocol inProt, final TProtocol outProt) throws TException { //populating request context ReqContext req_context = ReqContext.context(); TTransport trans = inProt.getTransport(); //Sasl transport TSaslServerTransport saslTrans = (TSaslServerTransport)trans; //remote address TSocket tsocket = (TSocket)saslTrans.getUnderlyingTransport(); Socket socket = tsocket.getSocket(); req_context.setRemoteAddress(socket.getInetAddress()); //remote subject SaslServer saslServer = saslTrans.getSaslServer(); String authId = saslServer.getAuthorizationID(); Subject remoteUser = new Subject(); remoteUser.getPrincipals().add(new User(authId)); req_context.setSubject(remoteUser); //invoke service handler return wrapped.process(inProt, outProt); } } public static class User implements Principal { private final String name; public User(String name) { this.name = name; } /** * Get the full name of the user. */ public String getName() { return name; } @Override public boolean equals(Object o) { if (this == o) { return true; } else if (o == null || getClass() != o.getClass()) { return false; } else { return (name.equals(((User) o).name)); } } @Override public int hashCode() { return name.hashCode(); } @Override public String toString() { return name; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import java.io.IOException; import java.net.InetAddress; import java.net.Socket; import java.net.UnknownHostException; import java.util.Map; import javax.security.auth.login.Configuration; import org.apache.thrift7.TException; import org.apache.thrift7.TProcessor; import org.apache.thrift7.protocol.TBinaryProtocol; import org.apache.thrift7.protocol.TProtocol; import org.apache.thrift7.server.THsHaServer; import org.apache.thrift7.server.TServer; import org.apache.thrift7.transport.TFramedTransport; import org.apache.thrift7.transport.TMemoryInputTransport; import org.apache.thrift7.transport.TNonblockingServerSocket; import org.apache.thrift7.transport.TSocket; import org.apache.thrift7.transport.TTransport; import org.apache.thrift7.transport.TTransportException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Simple transport for Thrift plugin. * * This plugin is designed to be backward compatible with existing Storm code. */ public class SimpleTransportPlugin implements ITransportPlugin { protected Configuration login_conf; private static final Logger LOG = LoggerFactory.getLogger(SimpleTransportPlugin.class); /** * Invoked once immediately after construction * @param conf Storm configuration * @param login_conf login configuration */ public void prepare(Map storm_conf, Configuration login_conf) { this.login_conf = login_conf; } /** * We will let Thrift to apply default transport factory */ public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException { TNonblockingServerSocket serverTransport = new TNonblockingServerSocket(port); THsHaServer.Args server_args = new THsHaServer.Args(serverTransport). processor(new SimpleWrapProcessor(processor)). workerThreads(64). protocolFactory(new TBinaryProtocol.Factory()); //construct THsHaServer return new THsHaServer(server_args); } /** * Connect to the specified server via framed transport * @param transport The underlying Thrift transport. */ public TTransport connect(TTransport transport, String serverHost) throws TTransportException { //create a framed transport TTransport conn = new TFramedTransport(transport); //connect conn.open(); LOG.debug("Simple client transport has been established"); return conn; } /** * Processor that populate simple transport info into ReqContext, and then invoke a service handler */ private class SimpleWrapProcessor implements TProcessor { final TProcessor wrapped; SimpleWrapProcessor(TProcessor wrapped) { this.wrapped = wrapped; } public boolean process(final TProtocol inProt, final TProtocol outProt) throws TException { //populating request context ReqContext req_context = ReqContext.context(); TTransport trans = inProt.getTransport(); if (trans instanceof TMemoryInputTransport) { try { req_context.setRemoteAddress(InetAddress.getLocalHost()); } catch (UnknownHostException e) { throw new RuntimeException(e); } } else if (trans instanceof TSocket) { TSocket tsocket = (TSocket)trans; //remote address Socket socket = tsocket.getSocket(); req_context.setRemoteAddress(socket.getInetAddress()); } //anonymous user req_context.setSubject(null); //invoke service handler return wrapped.process(inProt, outProt); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import java.io.IOException; import java.util.Map; import javax.security.auth.login.Configuration; import org.apache.thrift7.protocol.TBinaryProtocol; import org.apache.thrift7.protocol.TProtocol; import org.apache.thrift7.transport.TSocket; import org.apache.thrift7.transport.TTransport; import org.apache.thrift7.transport.TTransportException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.utils.Utils; public class ThriftClient { private static final Logger LOG = LoggerFactory.getLogger(ThriftClient.class); private TTransport _transport; protected TProtocol _protocol; public ThriftClient(Map storm_conf, String host, int port) throws TTransportException { this(storm_conf, host, port, null); } public ThriftClient(Map storm_conf, String host, int port, Integer timeout) throws TTransportException { try { //locate login configuration Configuration login_conf = AuthUtils.GetConfiguration(storm_conf); //construct a transport plugin ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(storm_conf, login_conf); //create a socket with server if(host==null) { throw new IllegalArgumentException("host is not set"); } if(port<=0) { throw new IllegalArgumentException("invalid port: "+port); } TSocket socket = new TSocket(host, port); if(timeout!=null) { socket.setTimeout(timeout); } final TTransport underlyingTransport = socket; //establish client-server transport via plugin _transport = transportPlugin.connect(underlyingTransport, host); } catch (IOException ex) { throw new RuntimeException(ex); } _protocol = null; if (_transport != null) _protocol = new TBinaryProtocol(_transport); } public TTransport transport() { return _transport; } public void close() { _transport.close(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth; import java.util.Map; import javax.security.auth.login.Configuration; import org.apache.thrift7.TProcessor; import org.apache.thrift7.server.TServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.utils.Utils; public class ThriftServer { private static final Logger LOG = LoggerFactory.getLogger(ThriftServer.class); private Map _storm_conf; //storm configuration protected TProcessor _processor = null; private int _port = 0; private TServer _server = null; private Configuration _login_conf; public ThriftServer(Map storm_conf, TProcessor processor, int port) { try { _storm_conf = storm_conf; _processor = processor; _port = port; //retrieve authentication configuration _login_conf = AuthUtils.GetConfiguration(_storm_conf); } catch (Exception x) { LOG.error(x.getMessage(), x); } } public void stop() { if (_server != null) _server.stop(); } /** * Is ThriftServer listening to requests? * @return */ public boolean isServing() { if (_server == null) return false; return _server.isServing(); } public void serve() { try { //locate our thrift transport plugin ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(_storm_conf, _login_conf); //server _server = transportPlugin.getServer(_port, _processor); //start accepting requests _server.serve(); } catch (Exception ex) { LOG.error("ThriftServer is being stopped due to: " + ex, ex); if (_server != null) _server.stop(); Runtime.getRuntime().halt(1); //shutdown server process since we could not handle Thrift requests any more } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth.authorizer; import java.util.Map; import backtype.storm.Config; import backtype.storm.security.auth.IAuthorizer; import backtype.storm.security.auth.ReqContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * An authorization implementation that denies everything, for testing purposes */ public class DenyAuthorizer implements IAuthorizer { private static final Logger LOG = LoggerFactory.getLogger(DenyAuthorizer.class); /** * Invoked once immediately after construction * @param conf Storm configuration */ public void prepare(Map conf) { } /** * permit() method is invoked for each incoming Thrift request * @param contrext request context * @param operation operation name * @param topology_storm configuration of targeted topology * @return true if the request is authorized, false if reject */ public boolean permit(ReqContext context, String operation, Map topology_conf) { LOG.info("[req "+ context.requestID()+ "] Access " + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + " principal:"+ (context.principal() == null? "null" : context.principal()) +" op:"+operation + " topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)); return false; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth.authorizer; import java.util.Map; import backtype.storm.Config; import backtype.storm.security.auth.IAuthorizer; import backtype.storm.security.auth.ReqContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * A no-op authorization implementation that illustrate info available for authorization decisions. */ public class NoopAuthorizer implements IAuthorizer { private static final Logger LOG = LoggerFactory.getLogger(NoopAuthorizer.class); /** * Invoked once immediately after construction * @param conf Storm configuration */ public void prepare(Map conf) { } /** * permit() method is invoked for each incoming Thrift request * @param context request context includes info about * @param operation operation name * @param topology_storm configuration of targeted topology * @return true if the request is authorized, false if reject */ public boolean permit(ReqContext context, String operation, Map topology_conf) { LOG.info("[req "+ context.requestID()+ "] Access " + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + " principal:"+(context.principal() == null? "null" : context.principal()) +" op:"+ operation + " topoology:"+ topology_conf.get(Config.TOPOLOGY_NAME)); return true; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/digest/ClientCallbackHandler.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth.digest; import java.io.IOException; import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.callback.NameCallback; import javax.security.auth.callback.PasswordCallback; import javax.security.auth.callback.UnsupportedCallbackException; import javax.security.sasl.AuthorizeCallback; import javax.security.sasl.RealmCallback; import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.security.auth.AuthUtils; /** * client side callback handler. */ public class ClientCallbackHandler implements CallbackHandler { private static final String USERNAME = "username"; private static final String PASSWORD = "password"; private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class); private String _username = null; private String _password = null; /** * Constructor based on a JAAS configuration * * For digest, you should have a pair of user name and password defined. * * @param configuration * @throws IOException */ public ClientCallbackHandler(Configuration configuration) throws IOException { if (configuration == null) return; AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_CLIENT); if (configurationEntries == null) { String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_CLIENT + "' entry in this configuration: Client cannot start."; throw new IOException(errorMessage); } _password = ""; for(AppConfigurationEntry entry: configurationEntries) { if (entry.getOptions().get(USERNAME) != null) { _username = (String)entry.getOptions().get(USERNAME); } if (entry.getOptions().get(PASSWORD) != null) { _password = (String)entry.getOptions().get(PASSWORD); } } } /** * This method is invoked by SASL for authentication challenges * @param callbacks a collection of challenge callbacks */ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { for (Callback c : callbacks) { if (c instanceof NameCallback) { LOG.debug("name callback"); NameCallback nc = (NameCallback) c; nc.setName(_username); } else if (c instanceof PasswordCallback) { LOG.debug("password callback"); PasswordCallback pc = (PasswordCallback)c; if (_password != null) { pc.setPassword(_password.toCharArray()); } } else if (c instanceof AuthorizeCallback) { LOG.debug("authorization callback"); AuthorizeCallback ac = (AuthorizeCallback) c; String authid = ac.getAuthenticationID(); String authzid = ac.getAuthorizationID(); if (authid.equals(authzid)) { ac.setAuthorized(true); } else { ac.setAuthorized(false); } if (ac.isAuthorized()) { ac.setAuthorizedID(authzid); } } else if (c instanceof RealmCallback) { RealmCallback rc = (RealmCallback) c; ((RealmCallback) c).setText(rc.getDefaultText()); } else { throw new UnsupportedCallbackException(c); } } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth.digest; import java.io.IOException; import java.util.Map; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.login.Configuration; import org.apache.thrift7.transport.TSaslClientTransport; import org.apache.thrift7.transport.TSaslServerTransport; import org.apache.thrift7.transport.TTransport; import org.apache.thrift7.transport.TTransportException; import org.apache.thrift7.transport.TTransportFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.security.auth.AuthUtils; import backtype.storm.security.auth.SaslTransportPlugin; public class DigestSaslTransportPlugin extends SaslTransportPlugin { public static final String DIGEST = "DIGEST-MD5"; private static final Logger LOG = LoggerFactory.getLogger(DigestSaslTransportPlugin.class); protected TTransportFactory getServerTransportFactory() throws IOException { //create an authentication callback handler CallbackHandler serer_callback_handler = new ServerCallbackHandler(login_conf); //create a transport factory that will invoke our auth callback for digest TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory(); factory.addServerDefinition(DIGEST, AuthUtils.SERVICE, "localhost", null, serer_callback_handler); LOG.info("SASL DIGEST-MD5 transport factory will be used"); return factory; } public TTransport connect(TTransport transport, String serverHost) throws TTransportException, IOException { ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf); TSaslClientTransport wrapper_transport = new TSaslClientTransport(DIGEST, null, AuthUtils.SERVICE, serverHost, null, client_callback_handler, transport); wrapper_transport.open(); LOG.debug("SASL DIGEST-MD5 client transport has been established"); return wrapper_transport; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/auth/digest/ServerCallbackHandler.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.auth.digest; import java.io.IOException; import java.util.HashMap; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.callback.NameCallback; import javax.security.auth.callback.PasswordCallback; import javax.security.auth.callback.UnsupportedCallbackException; import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.Configuration; import javax.security.sasl.AuthorizeCallback; import javax.security.sasl.RealmCallback; import backtype.storm.security.auth.AuthUtils; /** * SASL server side collback handler */ public class ServerCallbackHandler implements CallbackHandler { private static final String USER_PREFIX = "user_"; private static final Logger LOG = LoggerFactory.getLogger(ServerCallbackHandler.class); private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; private String userName; private final Map credentials = new HashMap(); public ServerCallbackHandler(Configuration configuration) throws IOException { if (configuration==null) return; AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_SERVER); if (configurationEntries == null) { String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_SERVER+"' entry in this configuration: Server cannot start."; throw new IOException(errorMessage); } credentials.clear(); for(AppConfigurationEntry entry: configurationEntries) { Map options = entry.getOptions(); // Populate DIGEST-MD5 user -> password map with JAAS configuration entries from the "Server" section. // Usernames are distinguished from other options by prefixing the username with a "user_" prefix. for(Map.Entry pair : options.entrySet()) { String key = pair.getKey(); if (key.startsWith(USER_PREFIX)) { String userName = key.substring(USER_PREFIX.length()); credentials.put(userName,(String)pair.getValue()); } } } } public void handle(Callback[] callbacks) throws UnsupportedCallbackException { for (Callback callback : callbacks) { if (callback instanceof NameCallback) { handleNameCallback((NameCallback) callback); } else if (callback instanceof PasswordCallback) { handlePasswordCallback((PasswordCallback) callback); } else if (callback instanceof RealmCallback) { handleRealmCallback((RealmCallback) callback); } else if (callback instanceof AuthorizeCallback) { handleAuthorizeCallback((AuthorizeCallback) callback); } } } private void handleNameCallback(NameCallback nc) { LOG.debug("handleNameCallback"); userName = nc.getDefaultName(); nc.setName(nc.getDefaultName()); } private void handlePasswordCallback(PasswordCallback pc) { LOG.debug("handlePasswordCallback"); if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) { // superuser: use Java system property for password, if available. pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray()); } else if (credentials.containsKey(userName) ) { pc.setPassword(credentials.get(userName).toCharArray()); } else { LOG.warn("No password found for user: " + userName); } } private void handleRealmCallback(RealmCallback rc) { LOG.debug("handleRealmCallback: "+ rc.getDefaultText()); rc.setText(rc.getDefaultText()); } private void handleAuthorizeCallback(AuthorizeCallback ac) { String authenticationID = ac.getAuthenticationID(); LOG.debug("Successfully authenticated client: authenticationID=" + authenticationID); ac.setAuthorizedID(authenticationID); ac.setAuthorized(true); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/security/serialization/BlowfishTupleSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.security.serialization; import java.util.Map; import org.apache.commons.codec.binary.Hex; import org.apache.log4j.Logger; import javax.crypto.KeyGenerator; import javax.crypto.SecretKey; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.serializers.BlowfishSerializer; import backtype.storm.serialization.types.ListDelegateSerializer; import backtype.storm.utils.ListDelegate; import backtype.storm.Config; /** * Apply Blowfish encrption for tuple communication to bolts */ public class BlowfishTupleSerializer extends Serializer { /** * The secret key (if any) for data encryption by blowfish payload serialization factory (BlowfishSerializationFactory). * You should use in via "storm -c topology.tuple.serializer.blowfish.key=YOURKEY -c topology.tuple.serializer=backtype.storm.security.serialization.BlowfishTupleSerializer jar ...". */ public static String SECRET_KEY = "topology.tuple.serializer.blowfish.key"; private static final Logger LOG = Logger.getLogger(BlowfishTupleSerializer.class); private BlowfishSerializer _serializer; public BlowfishTupleSerializer(Kryo kryo, Map storm_conf) { String encryption_key = null; try { encryption_key = (String)storm_conf.get(SECRET_KEY); LOG.debug("Blowfish serializer being constructed ..."); if (encryption_key == null) { throw new RuntimeException("Blowfish encryption key not specified"); } byte[] bytes = Hex.decodeHex(encryption_key.toCharArray()); _serializer = new BlowfishSerializer(new ListDelegateSerializer(), bytes); } catch (org.apache.commons.codec.DecoderException ex) { throw new RuntimeException("Blowfish encryption key invalid", ex); } } @Override public void write(Kryo kryo, Output output, ListDelegate object) { _serializer.write(kryo, output, object); } @Override public ListDelegate read(Kryo kryo, Input input, Class type) { return (ListDelegate)_serializer.read(kryo, input, type); } /** * Produce a blowfish key to be used in "Storm jar" command */ public static void main(String[] args) { try{ KeyGenerator kgen = KeyGenerator.getInstance("Blowfish"); SecretKey skey = kgen.generateKey(); byte[] raw = skey.getEncoded(); String keyString = new String(Hex.encodeHex(raw)); System.out.println("storm -c "+SECRET_KEY+"="+keyString+" -c "+Config.TOPOLOGY_TUPLE_SERIALIZER+"="+BlowfishTupleSerializer.class.getName() + " ..." ); } catch (Exception ex) { LOG.error(ex.getMessage()); ex.printStackTrace(); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/DefaultKryoFactory.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.Config; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Serializer; import java.util.Map; public class DefaultKryoFactory implements IKryoFactory { public static class KryoSerializableDefault extends Kryo { boolean _override = false; public void overrideDefault(boolean value) { _override = value; } @Override public Serializer getDefaultSerializer(Class type) { if(_override) { return new SerializableSerializer(); } else { return super.getDefaultSerializer(type); } } } @Override public Kryo getKryo(Map conf) { KryoSerializableDefault k = new KryoSerializableDefault(); k.setRegistrationRequired(!((Boolean) conf.get(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION))); k.setReferences(false); return k; } @Override public void preRegister(Kryo k, Map conf) { } public void postRegister(Kryo k, Map conf) { ((KryoSerializableDefault)k).overrideDefault(true); } @Override public void postDecorate(Kryo k, Map conf) { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/IKryoDecorator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import com.esotericsoftware.kryo.Kryo; public interface IKryoDecorator { void decorate(Kryo k); } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/IKryoFactory.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import com.esotericsoftware.kryo.Kryo; import java.util.Map; /** * An interface that controls the Kryo instance used by Storm for serialization. * The lifecycle is: * * 1. The Kryo instance is constructed using getKryo * 2. Storm registers the default classes (e.g. arrays, lists, maps, etc.) * 3. Storm calls preRegister hook * 4. Storm registers all user-defined registrations through topology.kryo.register * 5. Storm calls postRegister hook * 6. Storm calls all user-defined decorators through topology.kryo.decorators * 7. Storm calls postDecorate hook */ public interface IKryoFactory { Kryo getKryo(Map conf); void preRegister(Kryo k, Map conf); void postRegister(Kryo k, Map conf); void postDecorate(Kryo k, Map conf); } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/ITupleDeserializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.tuple.Tuple; import java.io.IOException; public interface ITupleDeserializer { Tuple deserialize(byte[] ser); } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/ITupleSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.tuple.Tuple; public interface ITupleSerializer { byte[] serialize(Tuple tuple); // long crc32(Tuple tuple); } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/KryoTupleDeserializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.task.GeneralTopologyContext; import backtype.storm.tuple.MessageId; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.TupleImpl; import backtype.storm.utils.WritableUtils; import com.esotericsoftware.kryo.io.Input; import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; import java.util.List; import java.util.Map; public class KryoTupleDeserializer implements ITupleDeserializer { GeneralTopologyContext _context; KryoValuesDeserializer _kryo; SerializationFactory.IdDictionary _ids; Input _kryoInput; public KryoTupleDeserializer(final Map conf, final GeneralTopologyContext context) { _kryo = new KryoValuesDeserializer(conf); _context = context; _ids = new SerializationFactory.IdDictionary(context.getRawTopology()); _kryoInput = new Input(1); } public Tuple deserialize(byte[] ser) { try { _kryoInput.setBuffer(ser); int taskId = _kryoInput.readInt(true); int streamId = _kryoInput.readInt(true); String componentName = _context.getComponentId(taskId); String streamName = _ids.getStreamName(componentName, streamId); MessageId id = MessageId.deserialize(_kryoInput); List values = _kryo.deserializeFrom(_kryoInput); return new TupleImpl(_context, values, taskId, streamName, id); } catch(IOException e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/KryoTupleSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.task.GeneralTopologyContext; import backtype.storm.tuple.Tuple; import com.esotericsoftware.kryo.io.Output; import java.io.IOException; import java.util.Map; public class KryoTupleSerializer implements ITupleSerializer { KryoValuesSerializer _kryo; SerializationFactory.IdDictionary _ids; Output _kryoOut; public KryoTupleSerializer(final Map conf, final GeneralTopologyContext context) { _kryo = new KryoValuesSerializer(conf); _kryoOut = new Output(2000, 2000000000); _ids = new SerializationFactory.IdDictionary(context.getRawTopology()); } public byte[] serialize(Tuple tuple) { try { _kryoOut.clear(); _kryoOut.writeInt(tuple.getSourceTask(), true); _kryoOut.writeInt(_ids.getStreamId(tuple.getSourceComponent(), tuple.getSourceStreamId()), true); tuple.getMessageId().serialize(_kryoOut); _kryo.serializeInto(tuple.getValues(), _kryoOut); return _kryoOut.toBytes(); } catch (IOException e) { throw new RuntimeException(e); } } // public long crc32(Tuple tuple) { // try { // CRC32OutputStream hasher = new CRC32OutputStream(); // _kryo.serializeInto(tuple.getValues(), hasher); // return hasher.getValue(); // } catch (IOException e) { // throw new RuntimeException(e); // } // } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/KryoValuesDeserializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.utils.ListDelegate; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; public class KryoValuesDeserializer { Kryo _kryo; Input _kryoInput; public KryoValuesDeserializer(Map conf) { _kryo = SerializationFactory.getKryo(conf); _kryoInput = new Input(1); } public List deserializeFrom(Input input) { ListDelegate delegate = (ListDelegate) _kryo.readObject(input, ListDelegate.class); return delegate.getDelegate(); } public List deserialize(byte[] ser) throws IOException { _kryoInput.setBuffer(ser); return deserializeFrom(_kryoInput); } public Object deserializeObject(byte[] ser) throws IOException { _kryoInput.setBuffer(ser); return _kryo.readClassAndObject(_kryoInput); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/KryoValuesSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.utils.ListDelegate; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Output; import java.io.IOException; import java.util.List; import java.util.Map; public class KryoValuesSerializer { Kryo _kryo; ListDelegate _delegate; Output _kryoOut; public KryoValuesSerializer(Map conf) { _kryo = SerializationFactory.getKryo(conf); _delegate = new ListDelegate(); _kryoOut = new Output(2000, 2000000000); } public void serializeInto(List values, Output out) throws IOException { // this ensures that list of values is always written the same way, regardless // of whether it's a java collection or one of clojure's persistent collections // (which have different serializers) // Doing this lets us deserialize as ArrayList and avoid writing the class here _delegate.setDelegate(values); _kryo.writeObject(out, _delegate); } public byte[] serialize(List values) throws IOException { _kryoOut.clear(); serializeInto(values, _kryoOut); return _kryoOut.toBytes(); } public byte[] serializeObject(Object obj) { _kryoOut.clear(); _kryo.writeClassAndObject(_kryoOut, obj); return _kryoOut.toBytes(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/SerializableSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; public class SerializableSerializer extends Serializer { @Override public void write(Kryo kryo, Output output, Object object) { ByteArrayOutputStream bos = new ByteArrayOutputStream(); try { ObjectOutputStream oos = new ObjectOutputStream(bos); oos.writeObject(object); oos.flush(); } catch(IOException e) { throw new RuntimeException(e); } byte[] ser = bos.toByteArray(); output.writeInt(ser.length); output.writeBytes(ser); } @Override public Object read(Kryo kryo, Input input, Class c) { int len = input.readInt(); byte[] ser = new byte[len]; input.readBytes(ser); ByteArrayInputStream bis = new ByteArrayInputStream(ser); try { ObjectInputStream ois = new ObjectInputStream(bis); return ois.readObject(); } catch(Exception e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/SerializationFactory.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization; import backtype.storm.Config; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.StormTopology; import backtype.storm.serialization.types.ArrayListSerializer; import backtype.storm.serialization.types.ListDelegateSerializer; import backtype.storm.serialization.types.HashMapSerializer; import backtype.storm.serialization.types.HashSetSerializer; import backtype.storm.transactional.TransactionAttempt; import backtype.storm.tuple.Values; import backtype.storm.utils.ListDelegate; import backtype.storm.utils.Utils; import carbonite.JavaBridge; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.serializers.DefaultSerializers.BigIntegerSerializer; import java.math.BigInteger; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.TreeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SerializationFactory { public static final Logger LOG = LoggerFactory.getLogger(SerializationFactory.class); public static Kryo getKryo(Map conf) { IKryoFactory kryoFactory = (IKryoFactory) Utils.newInstance((String) conf.get(Config.TOPOLOGY_KRYO_FACTORY)); Kryo k = kryoFactory.getKryo(conf); k.register(byte[].class); /* tuple payload serializer is specified via configuration */ String payloadSerializerName = (String)conf.get(Config.TOPOLOGY_TUPLE_SERIALIZER); try { Class serializerClass = Class.forName(payloadSerializerName); Serializer serializer = resolveSerializerInstance(k, ListDelegate.class, serializerClass, conf); k.register(ListDelegate.class, serializer); } catch (ClassNotFoundException ex) { throw new RuntimeException(ex); } k.register(ArrayList.class, new ArrayListSerializer()); k.register(HashMap.class, new HashMapSerializer()); k.register(HashSet.class, new HashSetSerializer()); k.register(BigInteger.class, new BigIntegerSerializer()); k.register(TransactionAttempt.class); k.register(Values.class); k.register(backtype.storm.metric.api.IMetricsConsumer.DataPoint.class); k.register(backtype.storm.metric.api.IMetricsConsumer.TaskInfo.class); try { JavaBridge.registerPrimitives(k); JavaBridge.registerCollections(k); } catch(Exception e) { throw new RuntimeException(e); } Map registrations = normalizeKryoRegister(conf); kryoFactory.preRegister(k, conf); boolean skipMissing = (Boolean) conf.get(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS); for(String klassName: registrations.keySet()) { String serializerClassName = registrations.get(klassName); try { Class klass = Class.forName(klassName); Class serializerClass = null; if(serializerClassName!=null) serializerClass = Class.forName(serializerClassName); if(serializerClass == null) { k.register(klass); } else { k.register(klass, resolveSerializerInstance(k, klass, serializerClass, conf)); } } catch (ClassNotFoundException e) { if(skipMissing) { LOG.info("Could not find serialization or class for " + serializerClassName + ". Skipping registration..."); } else { throw new RuntimeException(e); } } } kryoFactory.postRegister(k, conf); if (conf.get(Config.TOPOLOGY_KRYO_DECORATORS) != null) { for(String klassName : (List)conf.get(Config.TOPOLOGY_KRYO_DECORATORS)) { try { Class klass = Class.forName(klassName); IKryoDecorator decorator = (IKryoDecorator)klass.newInstance(); decorator.decorate(k); } catch(ClassNotFoundException e) { if(skipMissing) { LOG.info("Could not find kryo decorator named " + klassName + ". Skipping registration..."); } else { throw new RuntimeException(e); } } catch(InstantiationException e) { throw new RuntimeException(e); } catch(IllegalAccessException e) { throw new RuntimeException(e); } } } kryoFactory.postDecorate(k, conf); return k; } public static class IdDictionary { Map> streamNametoId = new HashMap>(); Map> streamIdToName = new HashMap>(); public IdDictionary(StormTopology topology) { List componentNames = new ArrayList(topology.get_spouts().keySet()); componentNames.addAll(topology.get_bolts().keySet()); componentNames.addAll(topology.get_state_spouts().keySet()); for(String name: componentNames) { ComponentCommon common = Utils.getComponentCommon(topology, name); List streams = new ArrayList(common.get_streams().keySet()); streamNametoId.put(name, idify(streams)); streamIdToName.put(name, Utils.reverseMap(streamNametoId.get(name))); } } public int getStreamId(String component, String stream) { return streamNametoId.get(component).get(stream); } public String getStreamName(String component, int stream) { return streamIdToName.get(component).get(stream); } private static Map idify(List names) { Collections.sort(names); Map ret = new HashMap(); int i = 1; for(String name: names) { ret.put(name, i); i++; } return ret; } } private static Serializer resolveSerializerInstance(Kryo k, Class superClass, Class serializerClass, Map conf) { try { try { return serializerClass.getConstructor(Kryo.class, Class.class, Map.class).newInstance(k, superClass, conf); } catch (Exception ex1) { try { return serializerClass.getConstructor(Kryo.class, Class.class).newInstance(k, superClass); } catch (Exception ex2) { try { return serializerClass.getConstructor(Kryo.class, Map.class).newInstance(k, conf); } catch (Exception ex3) { try { return serializerClass.getConstructor(Kryo.class).newInstance(k); } catch (Exception ex4) { try { return serializerClass.getConstructor(Class.class, Map.class).newInstance(superClass, conf); } catch (Exception ex5) { try { return serializerClass.getConstructor(Class.class).newInstance(superClass); } catch (Exception ex6) { return serializerClass.newInstance(); } } } } } } } catch (Exception ex) { throw new IllegalArgumentException("Unable to create serializer \"" + serializerClass.getName() + "\" for class: " + superClass.getName(), ex); } } private static Map normalizeKryoRegister(Map conf) { // TODO: de-duplicate this logic with the code in nimbus Object res = conf.get(Config.TOPOLOGY_KRYO_REGISTER); if(res==null) return new TreeMap(); Map ret = new HashMap(); if(res instanceof Map) { ret = (Map) res; } else { for(Object o: (List) res) { if(o instanceof Map) { ret.putAll((Map) o); } else { ret.put((String) o, null); } } } //ensure always same order for registrations with TreeMap return new TreeMap(ret); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/types/ArrayListSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization.types; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.serializers.CollectionSerializer; import java.util.ArrayList; import java.util.Collection; public class ArrayListSerializer extends CollectionSerializer { @Override public Collection create(Kryo kryo, Input input, Class type) { return new ArrayList(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/types/HashMapSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization.types; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.serializers.MapSerializer; import java.util.HashMap; import java.util.Map; public class HashMapSerializer extends MapSerializer { @Override public Map create(Kryo kryo, Input input, Class type) { return new HashMap(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/types/HashSetSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization.types; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.serializers.CollectionSerializer; import java.util.Collection; import java.util.HashSet; public class HashSetSerializer extends CollectionSerializer { @Override public Collection create(Kryo kryo, Input input, Class type) { return new HashSet(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/serialization/types/ListDelegateSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.serialization.types; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.serializers.CollectionSerializer; import backtype.storm.utils.ListDelegate; import java.util.Collection; public class ListDelegateSerializer extends CollectionSerializer { @Override public Collection create(Kryo kryo, Input input, Class type) { return new ListDelegate(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/IMultiSchemableSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; public interface IMultiSchemableSpout { MultiScheme getScheme(); void setScheme(MultiScheme scheme); } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/ISchemableSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; public interface ISchemableSpout { Scheme getScheme(); void setScheme(Scheme scheme); } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/ISpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import backtype.storm.task.TopologyContext; import java.util.Map; import java.io.Serializable; /** * ISpout is the core interface for implementing spouts. A Spout is responsible * for feeding messages into the topology for processing. For every tuple emitted by * a spout, Storm will track the (potentially very large) DAG of tuples generated * based on a tuple emitted by the spout. When Storm detects that every tuple in * that DAG has been successfully processed, it will send an ack message to the Spout. * *

If a tuple fails to be fully process within the configured timeout for the * topology (see {@link backtype.storm.Config}), Storm will send a fail message to the spout * for the message.

* *

When a Spout emits a tuple, it can tag the tuple with a message id. The message id * can be any type. When Storm acks or fails a message, it will pass back to the * spout the same message id to identify which tuple it's referring to. If the spout leaves out * the message id, or sets it to null, then Storm will not track the message and the spout * will not receive any ack or fail callbacks for the message.

* *

Storm executes ack, fail, and nextTuple all on the same thread. This means that an implementor * of an ISpout does not need to worry about concurrency issues between those methods. However, it * also means that an implementor must ensure that nextTuple is non-blocking: otherwise * the method could block acks and fails that are pending to be processed.

*/ public interface ISpout extends Serializable { /** * Called when a task for this component is initialized within a worker on the cluster. * It provides the spout with the environment in which the spout executes. * *

This includes the:

* * @param conf The Storm configuration for this spout. This is the configuration provided to the topology merged in with cluster configuration on this machine. * @param context This object can be used to get information about this task's place within the topology, including the task id and component id of this task, input and output information, etc. * @param collector The collector is used to emit tuples from this spout. Tuples can be emitted at any time, including the open and close methods. The collector is thread-safe and should be saved as an instance variable of this spout object. */ void open(Map conf, TopologyContext context, SpoutOutputCollector collector); /** * Called when an ISpout is going to be shutdown. There is no guarentee that close * will be called, because the supervisor kill -9's worker processes on the cluster. * *

The one context where close is guaranteed to be called is a topology is * killed when running Storm in local mode.

*/ void close(); /** * Called when a spout has been activated out of a deactivated mode. * nextTuple will be called on this spout soon. A spout can become activated * after having been deactivated when the topology is manipulated using the * `storm` client. */ void activate(); /** * Called when a spout has been deactivated. nextTuple will not be called while * a spout is deactivated. The spout may or may not be reactivated in the future. */ void deactivate(); /** * When this method is called, Storm is requesting that the Spout emit tuples to the * output collector. This method should be non-blocking, so if the Spout has no tuples * to emit, this method should return. nextTuple, ack, and fail are all called in a tight * loop in a single thread in the spout task. When there are no tuples to emit, it is courteous * to have nextTuple sleep for a short amount of time (like a single millisecond) * so as not to waste too much CPU. */ void nextTuple(); /** * Storm has determined that the tuple emitted by this spout with the msgId identifier * has been fully processed. Typically, an implementation of this method will take that * message off the queue and prevent it from being replayed. */ void ack(Object msgId); /** * The tuple emitted by this spout with the msgId identifier has failed to be * fully processed. Typically, an implementation of this method will put that * message back on the queue to be replayed at a later time. */ void fail(Object msgId); } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/ISpoutOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import java.util.List; public interface ISpoutOutputCollector { /** Returns the task ids that received the tuples. */ List emit(String streamId, List tuple, Object messageId); void emitDirect(int taskId, String streamId, List tuple, Object messageId); void reportError(Throwable error); } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/ISpoutWaitStrategy.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import java.util.Map; /** * The strategy a spout needs to use when its waiting. Waiting is * triggered in one of two conditions: * * 1. nextTuple emits no tuples * 2. The spout has hit maxSpoutPending and can't emit any more tuples * * The default strategy sleeps for one millisecond. */ public interface ISpoutWaitStrategy { void prepare(Map conf); void emptyEmit(long streak); } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/MultiScheme.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import java.util.List; import java.io.Serializable; import backtype.storm.tuple.Fields; public interface MultiScheme extends Serializable { public Iterable> deserialize(byte[] ser); public Fields getOutputFields(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/NothingEmptyEmitStrategy.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import java.util.Map; public class NothingEmptyEmitStrategy implements ISpoutWaitStrategy { @Override public void emptyEmit(long streak) { } @Override public void prepare(Map conf) { throw new UnsupportedOperationException("Not supported yet."); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/RawMultiScheme.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import java.util.List; import backtype.storm.tuple.Fields; import static backtype.storm.utils.Utils.tuple; import static java.util.Arrays.asList; public class RawMultiScheme implements MultiScheme { @Override public Iterable> deserialize(byte[] ser) { return asList(tuple(ser)); } @Override public Fields getOutputFields() { return new Fields("bytes"); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/RawScheme.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import backtype.storm.tuple.Fields; import java.util.List; import static backtype.storm.utils.Utils.tuple; public class RawScheme implements Scheme { public List deserialize(byte[] ser) { return tuple(ser); } public Fields getOutputFields() { return new Fields("bytes"); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/Scheme.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import backtype.storm.tuple.Fields; import java.io.Serializable; import java.util.List; public interface Scheme extends Serializable { public List deserialize(byte[] ser); public Fields getOutputFields(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/SchemeAsMultiScheme.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import java.util.Arrays; import java.util.List; import backtype.storm.tuple.Fields; public class SchemeAsMultiScheme implements MultiScheme { public final Scheme scheme; public SchemeAsMultiScheme(Scheme scheme) { this.scheme = scheme; } @Override public Iterable> deserialize(final byte[] ser) { List o = scheme.deserialize(ser); if(o == null) return null; else return Arrays.asList(o); } @Override public Fields getOutputFields() { return scheme.getOutputFields(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/ShellSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import backtype.storm.generated.ShellComponent; import backtype.storm.task.TopologyContext; import backtype.storm.utils.ShellProcess; import backtype.storm.utils.Utils; import java.util.Map; import java.util.List; import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.json.simple.JSONObject; public class ShellSpout implements ISpout { public static Logger LOG = LoggerFactory.getLogger(ShellSpout.class); private SpoutOutputCollector _collector; private String[] _command; private ShellProcess _process; public ShellSpout(ShellComponent component) { this(component.get_execution_command(), component.get_script()); } public ShellSpout(String... command) { _command = command; } public void open(Map stormConf, TopologyContext context, SpoutOutputCollector collector) { _process = new ShellProcess(_command); _collector = collector; try { Number subpid = _process.launch(stormConf, context); LOG.info("Launched subprocess with pid " + subpid); } catch (IOException e) { throw new RuntimeException("Error when launching multilang subprocess\n" + _process.getErrorsString(), e); } } public void close() { _process.destroy(); } private JSONObject _next; public void nextTuple() { if (_next == null) { _next = new JSONObject(); _next.put("command", "next"); } querySubprocess(_next); } private JSONObject _ack; public void ack(Object msgId) { if (_ack == null) { _ack = new JSONObject(); _ack.put("command", "ack"); } _ack.put("id", msgId); querySubprocess(_ack); } private JSONObject _fail; public void fail(Object msgId) { if (_fail == null) { _fail = new JSONObject(); _fail.put("command", "fail"); } _fail.put("id", msgId); querySubprocess(_fail); } private void querySubprocess(Object query) { try { _process.writeMessage(query); while (true) { JSONObject action = _process.readMessage(); String command = (String) action.get("command"); if (command.equals("sync")) { return; } else if (command.equals("log")) { String msg = (String) action.get("msg"); LOG.info("Shell msg: " + msg); } else if (command.equals("emit")) { String stream = (String) action.get("stream"); if (stream == null) stream = Utils.DEFAULT_STREAM_ID; Long task = (Long) action.get("task"); List tuple = (List) action.get("tuple"); Object messageId = (Object) action.get("id"); if (task == null) { List outtasks = _collector.emit(stream, tuple, messageId); Object need_task_ids = action.get("need_task_ids"); if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) { _process.writeMessage(outtasks); } } else { _collector.emitDirect((int)task.longValue(), stream, tuple, messageId); } } } } catch (IOException e) { throw new RuntimeException(e); } } @Override public void activate() { } @Override public void deactivate() { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/SleepSpoutWaitStrategy.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import backtype.storm.Config; import java.util.Map; public class SleepSpoutWaitStrategy implements ISpoutWaitStrategy { long sleepMillis; @Override public void prepare(Map conf) { sleepMillis = ((Number) conf.get(Config.TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS)).longValue(); } @Override public void emptyEmit(long streak) { try { Thread.sleep(sleepMillis); } catch (InterruptedException e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/spout/SpoutOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.spout; import backtype.storm.task.OutputCollector; import backtype.storm.utils.Utils; import java.util.List; /** * This output collector exposes the API for emitting tuples from an {@link backtype.storm.topology.IRichSpout}. * The main difference between this output collector and {@link OutputCollector} * for {@link backtype.storm.topology.IRichBolt} is that spouts can tag messages with ids so that they can be * acked or failed later on. This is the Spout portion of Storm's API to * guarantee that each message is fully processed at least once. */ public class SpoutOutputCollector implements ISpoutOutputCollector { ISpoutOutputCollector _delegate; public SpoutOutputCollector(ISpoutOutputCollector delegate) { _delegate = delegate; } /** * Emits a new tuple to the specified output stream with the given message ID. * When Storm detects that this tuple has been fully processed, or has failed * to be fully processed, the spout will receive an ack or fail callback respectively * with the messageId as long as the messageId was not null. If the messageId was null, * Storm will not track the tuple and no callback will be received. The emitted values must be * immutable. * * @return the list of task ids that this tuple was sent to */ public List emit(String streamId, List tuple, Object messageId) { return _delegate.emit(streamId, tuple, messageId); } /** * Emits a new tuple to the default output stream with the given message ID. * When Storm detects that this tuple has been fully processed, or has failed * to be fully processed, the spout will receive an ack or fail callback respectively * with the messageId as long as the messageId was not null. If the messageId was null, * Storm will not track the tuple and no callback will be received. The emitted values must be * immutable. * * @return the list of task ids that this tuple was sent to */ public List emit(List tuple, Object messageId) { return emit(Utils.DEFAULT_STREAM_ID, tuple, messageId); } /** * Emits a tuple to the default output stream with a null message id. Storm will * not track this message so ack and fail will never be called for this tuple. The * emitted values must be immutable. */ public List emit(List tuple) { return emit(tuple, null); } /** * Emits a tuple to the specified output stream with a null message id. Storm will * not track this message so ack and fail will never be called for this tuple. The * emitted values must be immutable. */ public List emit(String streamId, List tuple) { return emit(streamId, tuple, null); } /** * Emits a tuple to the specified task on the specified output stream. This output * stream must have been declared as a direct stream, and the specified task must * use a direct grouping on this stream to receive the message. The emitted values must be * immutable. */ public void emitDirect(int taskId, String streamId, List tuple, Object messageId) { _delegate.emitDirect(taskId, streamId, tuple, messageId); } /** * Emits a tuple to the specified task on the default output stream. This output * stream must have been declared as a direct stream, and the specified task must * use a direct grouping on this stream to receive the message. The emitted values must be * immutable. */ public void emitDirect(int taskId, List tuple, Object messageId) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple, messageId); } /** * Emits a tuple to the specified task on the specified output stream. This output * stream must have been declared as a direct stream, and the specified task must * use a direct grouping on this stream to receive the message. The emitted values must be * immutable. * *

Because no message id is specified, Storm will not track this message * so ack and fail will never be called for this tuple.

*/ public void emitDirect(int taskId, String streamId, List tuple) { emitDirect(taskId, streamId, tuple, null); } /** * Emits a tuple to the specified task on the default output stream. This output * stream must have been declared as a direct stream, and the specified task must * use a direct grouping on this stream to receive the message. The emitted values must be * immutable. * *

Because no message id is specified, Storm will not track this message * so ack and fail will never be called for this tuple.

*/ public void emitDirect(int taskId, List tuple) { emitDirect(taskId, tuple, null); } @Override public void reportError(Throwable error) { _delegate.reportError(error); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/state/IStateSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.state; import backtype.storm.task.TopologyContext; import java.io.Serializable; import java.util.Map; public interface IStateSpout extends Serializable { void open(Map conf, TopologyContext context); void close(); void nextTuple(StateSpoutOutputCollector collector); void synchronize(SynchronizeOutputCollector collector); } ================================================ FILE: storm-core/src/jvm/backtype/storm/state/IStateSpoutOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.state; public interface IStateSpoutOutputCollector extends ISynchronizeOutputCollector { void remove(int streamId, Object id); } ================================================ FILE: storm-core/src/jvm/backtype/storm/state/ISubscribedState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.state; import backtype.storm.tuple.Tuple; public interface ISubscribedState { void set(Object id, Tuple tuple); void remove(Object id); } ================================================ FILE: storm-core/src/jvm/backtype/storm/state/ISynchronizeOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.state; import java.util.List; public interface ISynchronizeOutputCollector { void add(int streamId, Object id, List tuple); } ================================================ FILE: storm-core/src/jvm/backtype/storm/state/StateSpoutOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.state; public class StateSpoutOutputCollector extends SynchronizeOutputCollector implements IStateSpoutOutputCollector { @Override public void remove(int streamId, Object id) { throw new UnsupportedOperationException("Not supported yet."); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/state/SynchronizeOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.state; import java.util.List; public class SynchronizeOutputCollector implements ISynchronizeOutputCollector { @Override public void add(int streamId, Object id, List tuple) { throw new UnsupportedOperationException("Not supported yet."); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/GeneralTopologyContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.Config; import backtype.storm.Constants; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.generated.StormTopology; import backtype.storm.tuple.Fields; import backtype.storm.utils.ThriftTopologyUtils; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import org.json.simple.JSONValue; import org.json.simple.JSONAware; public class GeneralTopologyContext implements JSONAware { private StormTopology _topology; private Map _taskToComponent; private Map> _componentToTasks; private Map> _componentToStreamToFields; private String _stormId; protected Map _stormConf; // pass in componentToSortedTasks for the case of running tons of tasks in single executor public GeneralTopologyContext(StormTopology topology, Map stormConf, Map taskToComponent, Map> componentToSortedTasks, Map> componentToStreamToFields, String stormId) { _topology = topology; _stormConf = stormConf; _taskToComponent = taskToComponent; _stormId = stormId; _componentToTasks = componentToSortedTasks; _componentToStreamToFields = componentToStreamToFields; } /** * Gets the unique id assigned to this topology. The id is the storm name with a * unique nonce appended to it. * @return the storm id */ public String getStormId() { return _stormId; } /** * Gets the Thrift object representing the topology. * * @return the Thrift definition representing the topology */ public StormTopology getRawTopology() { return _topology; } /** * Gets the component id for the specified task id. The component id maps * to a component id specified for a Spout or Bolt in the topology definition. * * @param taskId the task id * @return the component id for the input task id */ public String getComponentId(int taskId) { if(taskId==Constants.SYSTEM_TASK_ID) { return Constants.SYSTEM_COMPONENT_ID; } else { return _taskToComponent.get(taskId); } } /** * Gets the set of streams declared for the specified component. */ public Set getComponentStreams(String componentId) { return getComponentCommon(componentId).get_streams().keySet(); } /** * Gets the task ids allocated for the given component id. The task ids are * always returned in ascending order. */ public List getComponentTasks(String componentId) { List ret = _componentToTasks.get(componentId); if(ret==null) return new ArrayList(); else return new ArrayList(ret); } /** * Gets the declared output fields for the specified component/stream. */ public Fields getComponentOutputFields(String componentId, String streamId) { Fields ret = _componentToStreamToFields.get(componentId).get(streamId); if(ret==null) { throw new IllegalArgumentException("No output fields defined for component:stream " + componentId + ":" + streamId); } return ret; } /** * Gets the declared output fields for the specified global stream id. */ public Fields getComponentOutputFields(GlobalStreamId id) { return getComponentOutputFields(id.get_componentId(), id.get_streamId()); } /** * Gets the declared inputs to the specified component. * * @return A map from subscribed component/stream to the grouping subscribed with. */ public Map getSources(String componentId) { return getComponentCommon(componentId).get_inputs(); } /** * Gets information about who is consuming the outputs of the specified component, * and how. * * @return Map from stream id to component id to the Grouping used. */ public Map> getTargets(String componentId) { Map> ret = new HashMap>(); for(String otherComponentId: getComponentIds()) { Map inputs = getComponentCommon(otherComponentId).get_inputs(); for(GlobalStreamId id: inputs.keySet()) { if(id.get_componentId().equals(componentId)) { Map curr = ret.get(id.get_streamId()); if(curr==null) curr = new HashMap(); curr.put(otherComponentId, inputs.get(id)); ret.put(id.get_streamId(), curr); } } } return ret; } @Override public String toJSONString() { Map obj = new HashMap(); obj.put("task->component", _taskToComponent); // TODO: jsonify StormTopology // at the minimum should send source info return JSONValue.toJSONString(obj); } /** * Gets a map from task id to component id. */ public Map getTaskToComponent() { return _taskToComponent; } /** * Gets a list of all component ids in this topology */ public Set getComponentIds() { return ThriftTopologyUtils.getComponentIds(getRawTopology()); } public ComponentCommon getComponentCommon(String componentId) { return ThriftTopologyUtils.getComponentCommon(getRawTopology(), componentId); } public int maxTopologyMessageTimeout() { Integer max = Utils.getInt(_stormConf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)); for(String spout: getRawTopology().get_spouts().keySet()) { ComponentCommon common = getComponentCommon(spout); String jsonConf = common.get_json_conf(); if(jsonConf!=null) { Map conf = (Map) JSONValue.parse(jsonConf); Object comp = conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS); if(comp!=null) { max = Math.max(Utils.getInt(comp), max); } } } return max; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/IBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.tuple.Tuple; import java.util.Map; import java.io.Serializable; /** * An IBolt represents a component that takes tuples as input and produces tuples * as output. An IBolt can do everything from filtering to joining to functions * to aggregations. It does not have to process a tuple immediately and may * hold onto tuples to process later. * *

A bolt's lifecycle is as follows:

* *

IBolt object created on client machine. The IBolt is serialized into the topology * (using Java serialization) and submitted to the master machine of the cluster (Nimbus). * Nimbus then launches workers which deserialize the object, call prepare on it, and then * start processing tuples.

* *

If you want to parameterize an IBolt, you should set the parameter's through its * constructor and save the parameterization state as instance variables (which will * then get serialized and shipped to every task executing this bolt across the cluster).

* *

When defining bolts in Java, you should use the IRichBolt interface which adds * necessary methods for using the Java TopologyBuilder API.

*/ public interface IBolt extends Serializable { /** * Called when a task for this component is initialized within a worker on the cluster. * It provides the bolt with the environment in which the bolt executes. * *

This includes the:

* * @param stormConf The Storm configuration for this bolt. This is the configuration provided to the topology merged in with cluster configuration on this machine. * @param context This object can be used to get information about this task's place within the topology, including the task id and component id of this task, input and output information, etc. * @param collector The collector is used to emit tuples from this bolt. Tuples can be emitted at any time, including the prepare and cleanup methods. The collector is thread-safe and should be saved as an instance variable of this bolt object. */ void prepare(Map stormConf, TopologyContext context, OutputCollector collector); /** * Process a single tuple of input. The Tuple object contains metadata on it * about which component/stream/task it came from. The values of the Tuple can * be accessed using Tuple#getValue. The IBolt does not have to process the Tuple * immediately. It is perfectly fine to hang onto a tuple and process it later * (for instance, to do an aggregation or join). * *

Tuples should be emitted using the OutputCollector provided through the prepare method. * It is required that all input tuples are acked or failed at some point using the OutputCollector. * Otherwise, Storm will be unable to determine when tuples coming off the spouts * have been completed.

* *

For the common case of acking an input tuple at the end of the execute method, * see IBasicBolt which automates this.

* * @param input The input tuple to be processed. */ void execute(Tuple input); /** * Called when an IBolt is going to be shutdown. There is no guarentee that cleanup * will be called, because the supervisor kill -9's worker processes on the cluster. * *

The one context where cleanup is guaranteed to be called is when a topology * is killed when running Storm in local mode.

*/ void cleanup(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/IErrorReporter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; public interface IErrorReporter { void reportError(Throwable error); } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/IMetricsContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.metric.api.CombinedMetric; import backtype.storm.metric.api.ICombiner; import backtype.storm.metric.api.IMetric; import backtype.storm.metric.api.IReducer; import backtype.storm.metric.api.ReducedMetric; public interface IMetricsContext { T registerMetric(String name, T metric, int timeBucketSizeInSecs); ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs); CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs); } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/IOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.tuple.Tuple; import java.util.Collection; import java.util.List; public interface IOutputCollector extends IErrorReporter { /** * Returns the task ids that received the tuples. */ List emit(String streamId, Collection anchors, List tuple); void emitDirect(int taskId, String streamId, Collection anchors, List tuple); void ack(Tuple input); void fail(Tuple input); } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/OutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.tuple.Tuple; import backtype.storm.utils.Utils; import java.util.Arrays; import java.util.Collection; import java.util.List; /** * This output collector exposes the API for emitting tuples from an IRichBolt. * This is the core API for emitting tuples. For a simpler API, and a more restricted * form of stream processing, see IBasicBolt and BasicOutputCollector. */ public class OutputCollector implements IOutputCollector { private IOutputCollector _delegate; public OutputCollector(IOutputCollector delegate) { _delegate = delegate; } /** * Emits a new tuple to a specific stream with a single anchor. The emitted values must be * immutable. * * @param streamId the stream to emit to * @param anchor the tuple to anchor to * @param tuple the new output tuple from this bolt * @return the list of task ids that this new tuple was sent to */ public List emit(String streamId, Tuple anchor, List tuple) { return emit(streamId, Arrays.asList(anchor), tuple); } /** * Emits a new unanchored tuple to the specified stream. Because it's unanchored, * if a failure happens downstream, this new tuple won't affect whether any * spout tuples are considered failed or not. The emitted values must be * immutable. * * @param streamId the stream to emit to * @param tuple the new output tuple from this bolt * @return the list of task ids that this new tuple was sent to */ public List emit(String streamId, List tuple) { return emit(streamId, (List) null, tuple); } /** * Emits a new tuple to the default stream anchored on a group of input tuples. The emitted * values must be immutable. * * @param anchors the tuples to anchor to * @param tuple the new output tuple from this bolt * @return the list of task ids that this new tuple was sent to */ public List emit(Collection anchors, List tuple) { return emit(Utils.DEFAULT_STREAM_ID, anchors, tuple); } /** * Emits a new tuple to the default stream anchored on a single tuple. The emitted values must be * immutable. * * @param anchor the tuple to anchor to * @param tuple the new output tuple from this bolt * @return the list of task ids that this new tuple was sent to */ public List emit(Tuple anchor, List tuple) { return emit(Utils.DEFAULT_STREAM_ID, anchor, tuple); } /** * Emits a new unanchored tuple to the default stream. Beacuse it's unanchored, * if a failure happens downstream, this new tuple won't affect whether any * spout tuples are considered failed or not. The emitted values must be * immutable. * * @param tuple the new output tuple from this bolt * @return the list of task ids that this new tuple was sent to */ public List emit(List tuple) { return emit(Utils.DEFAULT_STREAM_ID, tuple); } /** * Emits a tuple directly to the specified task id on the specified stream. * If the target bolt does not subscribe to this bolt using a direct grouping, * the tuple will not be sent. If the specified output stream is not declared * as direct, or the target bolt subscribes with a non-direct grouping, * an error will occur at runtime. The emitted values must be * immutable. * * @param taskId the taskId to send the new tuple to * @param streamId the stream to send the tuple on. It must be declared as a direct stream in the topology definition. * @param anchor the tuple to anchor to * @param tuple the new output tuple from this bolt */ public void emitDirect(int taskId, String streamId, Tuple anchor, List tuple) { emitDirect(taskId, streamId, Arrays.asList(anchor), tuple); } /** * Emits a tuple directly to the specified task id on the specified stream. * If the target bolt does not subscribe to this bolt using a direct grouping, * the tuple will not be sent. If the specified output stream is not declared * as direct, or the target bolt subscribes with a non-direct grouping, * an error will occur at runtime. Note that this method does not use anchors, * so downstream failures won't affect the failure status of any spout tuples. * The emitted values must be immutable. * * @param taskId the taskId to send the new tuple to * @param streamId the stream to send the tuple on. It must be declared as a direct stream in the topology definition. * @param tuple the new output tuple from this bolt */ public void emitDirect(int taskId, String streamId, List tuple) { emitDirect(taskId, streamId, (List) null, tuple); } /** * Emits a tuple directly to the specified task id on the default stream. * If the target bolt does not subscribe to this bolt using a direct grouping, * the tuple will not be sent. If the specified output stream is not declared * as direct, or the target bolt subscribes with a non-direct grouping, * an error will occur at runtime. The emitted values must be * immutable. * *

The default stream must be declared as direct in the topology definition. * See OutputDeclarer#declare for how this is done when defining topologies * in Java.

* * @param taskId the taskId to send the new tuple to * @param anchosr the tuples to anchor to * @param tuple the new output tuple from this bolt */ public void emitDirect(int taskId, Collection anchors, List tuple) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchors, tuple); } /** * Emits a tuple directly to the specified task id on the default stream. * If the target bolt does not subscribe to this bolt using a direct grouping, * the tuple will not be sent. If the specified output stream is not declared * as direct, or the target bolt subscribes with a non-direct grouping, * an error will occur at runtime. The emitted values must be * immutable. * *

The default stream must be declared as direct in the topology definition. * See OutputDeclarer#declare for how this is done when defining topologies * in Java.

* * @param taskId the taskId to send the new tuple to * @param anchor the tuple to anchor to * @param tuple the new output tuple from this bolt */ public void emitDirect(int taskId, Tuple anchor, List tuple) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchor, tuple); } /** * Emits a tuple directly to the specified task id on the default stream. * If the target bolt does not subscribe to this bolt using a direct grouping, * the tuple will not be sent. If the specified output stream is not declared * as direct, or the target bolt subscribes with a non-direct grouping, * an error will occur at runtime. The emitted values must be * immutable. * *

The default stream must be declared as direct in the topology definition. * See OutputDeclarer#declare for how this is done when defining topologies * in Java.

* *

Note that this method does not use anchors, so downstream failures won't * affect the failure status of any spout tuples.

* * @param taskId the taskId to send the new tuple to * @param tuple the new output tuple from this bolt */ public void emitDirect(int taskId, List tuple) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); } @Override public List emit(String streamId, Collection anchors, List tuple) { return _delegate.emit(streamId, anchors, tuple); } @Override public void emitDirect(int taskId, String streamId, Collection anchors, List tuple) { _delegate.emitDirect(taskId, streamId, anchors, tuple); } @Override public void ack(Tuple input) { _delegate.ack(input); } @Override public void fail(Tuple input) { _delegate.fail(input); } @Override public void reportError(Throwable error) { _delegate.reportError(error); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/ShellBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.Config; import backtype.storm.generated.ShellComponent; import backtype.storm.tuple.MessageId; import backtype.storm.tuple.Tuple; import backtype.storm.utils.Utils; import backtype.storm.utils.ShellProcess; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; import static java.util.concurrent.TimeUnit.SECONDS; import java.util.List; import java.util.Map; import java.util.Random; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.json.simple.JSONObject; /** * A bolt that shells out to another process to process tuples. ShellBolt * communicates with that process over stdio using a special protocol. An ~100 * line library is required to implement that protocol, and adapter libraries * currently exist for Ruby and Python. * *

To run a ShellBolt on a cluster, the scripts that are shelled out to must be * in the resources directory within the jar submitted to the master. * During development/testing on a local machine, that resources directory just * needs to be on the classpath.

* *

When creating topologies using the Java API, subclass this bolt and implement * the IRichBolt interface to create components for the topology that use other languages. For example: *

* *
 * public class MyBolt extends ShellBolt implements IRichBolt {
 *      public MyBolt() {
 *          super("python", "mybolt.py");
 *      }
 *
 *      public void declareOutputFields(OutputFieldsDeclarer declarer) {
 *          declarer.declare(new Fields("field1", "field2"));
 *      }
 * }
 * 
*/ public class ShellBolt implements IBolt { public static Logger LOG = LoggerFactory.getLogger(ShellBolt.class); Process _subprocess; OutputCollector _collector; Map _inputs = new ConcurrentHashMap(); private String[] _command; private ShellProcess _process; private volatile boolean _running = true; private volatile Throwable _exception; private LinkedBlockingQueue _pendingWrites = new LinkedBlockingQueue(); private Random _rand; private Thread _readerThread; private Thread _writerThread; public ShellBolt(ShellComponent component) { this(component.get_execution_command(), component.get_script()); } public ShellBolt(String... command) { _command = command; } public void prepare(Map stormConf, TopologyContext context, final OutputCollector collector) { Object maxPending = stormConf.get(Config.TOPOLOGY_SHELLBOLT_MAX_PENDING); if (maxPending != null) { this._pendingWrites = new LinkedBlockingQueue(((Number)maxPending).intValue()); } _rand = new Random(); _process = new ShellProcess(_command); _collector = collector; try { //subprocesses must send their pid first thing Number subpid = _process.launch(stormConf, context); LOG.info("Launched subprocess with pid " + subpid); } catch (IOException e) { throw new RuntimeException("Error when launching multilang subprocess\n" + _process.getErrorsString(), e); } // reader _readerThread = new Thread(new Runnable() { public void run() { while (_running) { try { JSONObject action = _process.readMessage(); if (action == null) { // ignore sync } String command = (String) action.get("command"); if(command.equals("ack")) { handleAck(action); } else if (command.equals("fail")) { handleFail(action); } else if (command.equals("error")) { handleError(action); } else if (command.equals("log")) { String msg = (String) action.get("msg"); LOG.info("Shell msg: " + msg); } else if (command.equals("emit")) { handleEmit(action); } } catch (InterruptedException e) { } catch (Throwable t) { die(t); } } } }); _readerThread.start(); _writerThread = new Thread(new Runnable() { public void run() { while (_running) { try { Object write = _pendingWrites.poll(1, SECONDS); if (write != null) { _process.writeMessage(write); } // drain the error stream to avoid dead lock because of full error stream buffer _process.drainErrorStream(); } catch (InterruptedException e) { } catch (Throwable t) { die(t); } } } }); _writerThread.start(); } public void execute(Tuple input) { if (_exception != null) { throw new RuntimeException(_exception); } //just need an id String genId = Long.toString(_rand.nextLong()); _inputs.put(genId, input); try { JSONObject obj = new JSONObject(); obj.put("id", genId); obj.put("comp", input.getSourceComponent()); obj.put("stream", input.getSourceStreamId()); obj.put("task", input.getSourceTask()); obj.put("tuple", input.getValues()); _pendingWrites.put(obj); } catch(InterruptedException e) { throw new RuntimeException("Error during multilang processing", e); } } public void cleanup() { _running = false; _process.destroy(); _inputs.clear(); } private void handleAck(Map action) { String id = (String) action.get("id"); Tuple acked = _inputs.remove(id); if(acked==null) { throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id); } _collector.ack(acked); } private void handleFail(Map action) { String id = (String) action.get("id"); Tuple failed = _inputs.remove(id); if(failed==null) { throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id); } _collector.fail(failed); } private void handleError(Map action) { String msg = (String) action.get("msg"); _collector.reportError(new Exception("Shell Process Exception: " + msg)); } private void handleEmit(Map action) throws InterruptedException { String stream = (String) action.get("stream"); if(stream==null) stream = Utils.DEFAULT_STREAM_ID; Long task = (Long) action.get("task"); List tuple = (List) action.get("tuple"); List anchors = new ArrayList(); Object anchorObj = action.get("anchors"); if(anchorObj!=null) { if(anchorObj instanceof String) { anchorObj = Arrays.asList(anchorObj); } for(Object o: (List) anchorObj) { Tuple t = _inputs.get((String) o); if (t == null) { throw new RuntimeException("Anchored onto " + o + " after ack/fail"); } anchors.add(t); } } if(task==null) { List outtasks = _collector.emit(stream, anchors, tuple); Object need_task_ids = action.get("need_task_ids"); if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) { _pendingWrites.put(outtasks); } } else { _collector.emitDirect((int)task.longValue(), stream, anchors, tuple); } } private void die(Throwable exception) { _exception = exception; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/TopologyContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.generated.StormTopology; import backtype.storm.hooks.ITaskHook; import backtype.storm.metric.api.IMetric; import backtype.storm.metric.api.IReducer; import backtype.storm.metric.api.ICombiner; import backtype.storm.metric.api.ReducedMetric; import backtype.storm.metric.api.CombinedMetric; import backtype.storm.state.ISubscribedState; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.commons.lang.NotImplementedException; /** * A TopologyContext is given to bolts and spouts in their "prepare" and "open" * methods, respectively. This object provides information about the component's * place within the topology, such as task ids, inputs and outputs, etc. * *

The TopologyContext is also used to declare ISubscribedState objects to * synchronize state with StateSpouts this object is subscribed to.

*/ public class TopologyContext extends WorkerTopologyContext implements IMetricsContext { private Integer _taskId; private Map _taskData = new HashMap(); private List _hooks = new ArrayList(); private Map _executorData; private Map>> _registeredMetrics; private clojure.lang.Atom _openOrPrepareWasCalled; public TopologyContext(StormTopology topology, Map stormConf, Map taskToComponent, Map> componentToSortedTasks, Map> componentToStreamToFields, String stormId, String codeDir, String pidDir, Integer taskId, Integer workerPort, List workerTasks, Map defaultResources, Map userResources, Map executorData, Map registeredMetrics, clojure.lang.Atom openOrPrepareWasCalled) { super(topology, stormConf, taskToComponent, componentToSortedTasks, componentToStreamToFields, stormId, codeDir, pidDir, workerPort, workerTasks, defaultResources, userResources); _taskId = taskId; _executorData = executorData; _registeredMetrics = registeredMetrics; _openOrPrepareWasCalled = openOrPrepareWasCalled; } /** * All state from all subscribed state spouts streams will be synced with * the provided object. * *

It is recommended that your ISubscribedState object is kept as an instance * variable of this object. The recommended usage of this method is as follows:

* *

* _myState = context.setAllSubscribedState(new MyState()); *

* @param obj Provided ISubscribedState implementation * @return Returns the ISubscribedState object provided */ public T setAllSubscribedState(T obj) { //check that only subscribed to one component/stream for statespout //setsubscribedstate appropriately throw new NotImplementedException(); } /** * Synchronizes the default stream from the specified state spout component * id with the provided ISubscribedState object. * *

The recommended usage of this method is as follows:

*

* _myState = context.setSubscribedState(componentId, new MyState()); *

* * @param componentId the id of the StateSpout component to subscribe to * @param obj Provided ISubscribedState implementation * @return Returns the ISubscribedState object provided */ public T setSubscribedState(String componentId, T obj) { return setSubscribedState(componentId, Utils.DEFAULT_STREAM_ID, obj); } /** * Synchronizes the specified stream from the specified state spout component * id with the provided ISubscribedState object. * *

The recommended usage of this method is as follows:

*

* _myState = context.setSubscribedState(componentId, streamId, new MyState()); *

* * @param componentId the id of the StateSpout component to subscribe to * @param streamId the stream to subscribe to * @param obj Provided ISubscribedState implementation * @return Returns the ISubscribedState object provided */ public T setSubscribedState(String componentId, String streamId, T obj) { throw new NotImplementedException(); } /** * Gets the task id of this task. * * @return the task id */ public int getThisTaskId() { return _taskId; } /** * Gets the component id for this task. The component id maps * to a component id specified for a Spout or Bolt in the topology definition. * @return */ public String getThisComponentId() { return getComponentId(_taskId); } /** * Gets the declared output fields for the specified stream id for the component * this task is a part of. */ public Fields getThisOutputFields(String streamId) { return getComponentOutputFields(getThisComponentId(), streamId); } /** * Gets the set of streams declared for the component of this task. */ public Set getThisStreams() { return getComponentStreams(getThisComponentId()); } /** * Gets the index of this task id in getComponentTasks(getThisComponentId()). * An example use case for this method is determining which task * accesses which resource in a distributed resource to ensure an even distribution. */ public int getThisTaskIndex() { List tasks = new ArrayList(getComponentTasks(getThisComponentId())); Collections.sort(tasks); for(int i=0; i getThisSources() { return getSources(getThisComponentId()); } /** * Gets information about who is consuming the outputs of this component, and how. * * @return Map from stream id to component id to the Grouping used. */ public Map> getThisTargets() { return getTargets(getThisComponentId()); } public void setTaskData(String name, Object data) { _taskData.put(name, data); } public Object getTaskData(String name) { return _taskData.get(name); } public void setExecutorData(String name, Object data) { _executorData.put(name, data); } public Object getExecutorData(String name) { return _executorData.get(name); } public void addTaskHook(ITaskHook hook) { hook.prepare(_stormConf, this); _hooks.add(hook); } public Collection getHooks() { return _hooks; } /* * Register a IMetric instance. * Storm will then call getValueAndReset on the metric every timeBucketSizeInSecs * and the returned value is sent to all metrics consumers. * You must call this during IBolt::prepare or ISpout::open. * @return The IMetric argument unchanged. */ public T registerMetric(String name, T metric, int timeBucketSizeInSecs) { if((Boolean)_openOrPrepareWasCalled.deref() == true) { throw new RuntimeException("TopologyContext.registerMetric can only be called from within overridden " + "IBolt::prepare() or ISpout::open() method."); } Map m1 = _registeredMetrics; if(!m1.containsKey(timeBucketSizeInSecs)) { m1.put(timeBucketSizeInSecs, new HashMap()); } Map m2 = (Map)m1.get(timeBucketSizeInSecs); if(!m2.containsKey(_taskId)) { m2.put(_taskId, new HashMap()); } Map m3 = (Map)m2.get(_taskId); if(m3.containsKey(name)) { throw new RuntimeException("The same metric name `" + name + "` was registered twice." ); } else { m3.put(name, metric); } return metric; } /* * Convinience method for registering ReducedMetric. */ public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) { return registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs); } /* * Convinience method for registering CombinedMetric. */ public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) { return registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/task/WorkerTopologyContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.task; import backtype.storm.generated.StormTopology; import backtype.storm.tuple.Fields; import java.io.File; import java.io.IOException; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; public class WorkerTopologyContext extends GeneralTopologyContext { public static final String SHARED_EXECUTOR = "executor"; private Integer _workerPort; private List _workerTasks; private String _codeDir; private String _pidDir; Map _userResources; Map _defaultResources; public WorkerTopologyContext( StormTopology topology, Map stormConf, Map taskToComponent, Map> componentToSortedTasks, Map> componentToStreamToFields, String stormId, String codeDir, String pidDir, Integer workerPort, List workerTasks, Map defaultResources, Map userResources ) { super(topology, stormConf, taskToComponent, componentToSortedTasks, componentToStreamToFields, stormId); _codeDir = codeDir; _defaultResources = defaultResources; _userResources = userResources; try { if(pidDir!=null) { _pidDir = new File(pidDir).getCanonicalPath(); } else { _pidDir = null; } } catch (IOException e) { throw new RuntimeException("Could not get canonical path for " + _pidDir, e); } _workerPort = workerPort; _workerTasks = workerTasks; } /** * Gets all the task ids that are running in this worker process * (including the task for this task). */ public List getThisWorkerTasks() { return _workerTasks; } public Integer getThisWorkerPort() { return _workerPort; } /** * Gets the location of the external resources for this worker on the * local filesystem. These external resources typically include bolts implemented * in other languages, such as Ruby or Python. */ public String getCodeDir() { return _codeDir; } /** * If this task spawns any subprocesses, those subprocesses must immediately * write their PID to this directory on the local filesystem to ensure that * Storm properly destroys that process when the worker is shutdown. */ public String getPIDDir() { return _pidDir; } public Object getResource(String name) { return _userResources.get(name); } public ExecutorService getSharedExecutor() { return (ExecutorService) _defaultResources.get(SHARED_EXECUTOR); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/AckFailDelegate.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import java.io.Serializable; public interface AckFailDelegate extends Serializable { public void ack(Object id); public void fail(Object id); } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/AckFailMapTracker.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.utils.RegisteredGlobalState; import java.util.HashSet; import java.util.Set; public class AckFailMapTracker implements AckFailDelegate { String _acked; String _failed; public AckFailMapTracker() { _acked = RegisteredGlobalState.registerState(new HashSet()); _failed = RegisteredGlobalState.registerState(new HashSet()); } public boolean isAcked(Object id) { return ((Set)RegisteredGlobalState.getState(_acked)).contains(id); } public boolean isFailed(Object id) { return ((Set)RegisteredGlobalState.getState(_failed)).contains(id); } @Override public void ack(Object id) { ((Set)RegisteredGlobalState.getState(_acked)).add(id); } @Override public void fail(Object id) { ((Set)RegisteredGlobalState.getState(_failed)).add(id); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/AckTracker.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; public class AckTracker implements AckFailDelegate { private static Map acks = new ConcurrentHashMap(); private String _id; public AckTracker() { _id = UUID.randomUUID().toString(); acks.put(_id, new AtomicInteger(0)); } @Override public void ack(Object id) { acks.get(_id).incrementAndGet(); } @Override public void fail(Object id) { } public int getNumAcks() { return acks.get(_id).intValue(); } public void resetNumAcks() { acks.get(_id).set(0); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/BatchNumberList.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBatchBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; public class BatchNumberList extends BaseBatchBolt { @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("word", "list")); } String _wordComponent; public BatchNumberList(String wordComponent) { _wordComponent = wordComponent; } String word = null; List intSet = new ArrayList(); BatchOutputCollector _collector; @Override public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { _collector = collector; } @Override public void execute(Tuple tuple) { if(tuple.getSourceComponent().equals(_wordComponent)) { this.word = tuple.getString(1); } else { intSet.add(tuple.getInteger(1)); } } @Override public void finishBatch() { if(word!=null) { Collections.sort(intSet); _collector.emit(new Values(word, intSet)); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/BatchProcessWord.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.BasicOutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBasicBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; public class BatchProcessWord extends BaseBasicBolt { @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("id", "size")); } @Override public void execute(Tuple input, BasicOutputCollector collector) { collector.emit(new Values(input.getValue(0), input.getString(1).length())); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/BatchRepeatA.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.BasicOutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBasicBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; public class BatchRepeatA extends BaseBasicBolt { @Override public void execute(Tuple input, BasicOutputCollector collector) { Object id = input.getValue(0); String word = input.getString(1); for(int i=0; i getComponentConfiguration() { return new HashMap(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/CompleteTopologyParam.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.Config; /** * The param class for the Testing.completeTopology. */ public class CompleteTopologyParam { /** * The mocked spout sources */ private MockedSources mockedSources; /** * the config for the topology when it was submitted to the cluster */ private Config stormConf; /** * whether cleanup the state? */ private Boolean cleanupState; /** * the topology name you want to submit to the cluster */ private String topologyName; public MockedSources getMockedSources() { return mockedSources; } public void setMockedSources(MockedSources mockedSources) { this.mockedSources = mockedSources; } public Config getStormConf() { return stormConf; } public void setStormConf(Config stormConf) { this.stormConf = stormConf; } public Boolean getCleanupState() { return cleanupState; } public void setCleanupState(Boolean cleanupState) { this.cleanupState = cleanupState; } public String getTopologyName() { return topologyName; } public void setTopologyName(String topologyName) { this.topologyName = topologyName; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/CountingBatchBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBatchBolt; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import java.util.Map; public class CountingBatchBolt extends BaseBatchBolt { BatchOutputCollector _collector; Object _id; int _count = 0; @Override public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { _collector = collector; _id = id; } @Override public void execute(Tuple tuple) { _count++; } @Override public void finishBatch() { _collector.emit(new Values(_id, _count)); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("tx", "count")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/CountingCommitBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.topology.base.BaseTransactionalBolt; import backtype.storm.transactional.ICommitter; import backtype.storm.transactional.TransactionAttempt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import java.util.Map; public class CountingCommitBolt extends BaseTransactionalBolt implements ICommitter { BatchOutputCollector _collector; TransactionAttempt _id; int _count = 0; @Override public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt id) { _id = id; _collector = collector; } @Override public void execute(Tuple tuple) { _count++; } @Override public void finishBatch() { _collector.emit(new Values(_id, _count)); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("tx", "count")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/FeederSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.OutputFieldsDeclarer; import java.util.Map; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import backtype.storm.utils.InprocMessaging; import java.util.HashMap; import java.util.List; import java.util.UUID; public class FeederSpout extends BaseRichSpout { private int _id; private Fields _outFields; private SpoutOutputCollector _collector; private AckFailDelegate _ackFailDelegate; public FeederSpout(Fields outFields) { _id = InprocMessaging.acquireNewPort(); _outFields = outFields; } public void setAckFailDelegate(AckFailDelegate d) { _ackFailDelegate = d; } public void feed(List tuple) { feed(tuple, UUID.randomUUID().toString()); } public void feed(List tuple, Object msgId) { InprocMessaging.sendMessage(_id, new Values(tuple, msgId)); } public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _collector = collector; } public void close() { } public void nextTuple() { List toEmit = (List) InprocMessaging.pollMessage(_id); if(toEmit!=null) { List tuple = (List) toEmit.get(0); Object msgId = toEmit.get(1); _collector.emit(tuple, msgId); } else { try { Thread.sleep(1); } catch (InterruptedException e) { throw new RuntimeException(e); } } } public void ack(Object msgId) { if(_ackFailDelegate!=null) { _ackFailDelegate.ack(msgId); } } public void fail(Object msgId) { if(_ackFailDelegate!=null) { _ackFailDelegate.fail(msgId); } } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(_outFields); } @Override public Map getComponentConfiguration() { return new HashMap(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/FixedTuple.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.utils.Utils; import java.io.Serializable; import java.util.List; public class FixedTuple implements Serializable { public String stream; public List values; public FixedTuple(List values) { this.stream = Utils.DEFAULT_STREAM_ID; this.values = values; } public FixedTuple(String stream, List values) { this.stream = stream; this.values = values; } @Override public String toString() { return stream + ":" + "<" + values.toString() + ">"; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/FixedTupleSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichSpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; import static backtype.storm.utils.Utils.get; public class FixedTupleSpout implements IRichSpout { private static final Map acked = new HashMap(); private static final Map failed = new HashMap(); public static int getNumAcked(String stormId) { synchronized(acked) { return get(acked, stormId, 0); } } public static int getNumFailed(String stormId) { synchronized(failed) { return get(failed, stormId, 0); } } public static void clear(String stormId) { acked.remove(stormId); failed.remove(stormId); } private List _tuples; private SpoutOutputCollector _collector; private TopologyContext _context; private List _serveTuples; private Map _pending; private String _id; private String _fieldName; public FixedTupleSpout(List tuples) { this(tuples, null); } public FixedTupleSpout(List tuples, String fieldName) { _id = UUID.randomUUID().toString(); synchronized(acked) { acked.put(_id, 0); } synchronized(failed) { failed.put(_id, 0); } _tuples = new ArrayList(); for(Object o: tuples) { FixedTuple ft; if(o instanceof FixedTuple) { ft = (FixedTuple) o; } else { ft = new FixedTuple((List) o); } _tuples.add(ft); } _fieldName = fieldName; } public List getSourceTuples() { return _tuples; } public int getCompleted() { int ackedAmt; int failedAmt; synchronized(acked) { ackedAmt = acked.get(_id); } synchronized(failed) { failedAmt = failed.get(_id); } return ackedAmt + failedAmt; } public void cleanup() { synchronized(acked) { acked.remove(_id); } synchronized(failed) { failed.remove(_id); } } public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _context = context; List tasks = context.getComponentTasks(context.getThisComponentId()); int startIndex; for(startIndex=0; startIndex(); _serveTuples = new ArrayList(); for(int i=startIndex; i<_tuples.size(); i+=tasks.size()) { _serveTuples.add(_tuples.get(i)); } } public void close() { } public void nextTuple() { if(_serveTuples.size()>0) { FixedTuple ft = _serveTuples.remove(0); String id = UUID.randomUUID().toString(); _pending.put(id, ft); _collector.emit(ft.stream, ft.values, id); } else { Utils.sleep(100); } } public void ack(Object msgId) { synchronized(acked) { int curr = get(acked, _id, 0); acked.put(_id, curr+1); } } public void fail(Object msgId) { synchronized(failed) { int curr = get(failed, _id, 0); failed.put(_id, curr+1); } } @Override public void activate() { } @Override public void deactivate() { } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { if (_fieldName != null) { declarer.declare(new Fields(_fieldName)); } } @Override public Map getComponentConfiguration() { return null; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/IdentityBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.BasicOutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBasicBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; public class IdentityBolt extends BaseBasicBolt { Fields _fields; public IdentityBolt(Fields fields) { _fields = fields; } @Override public void execute(Tuple input, BasicOutputCollector collector) { collector.emit(input.getValues()); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(_fields); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/KeyedCountingBatchBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBatchBolt; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import backtype.storm.utils.Utils; import java.util.HashMap; import java.util.Map; public class KeyedCountingBatchBolt extends BaseBatchBolt { BatchOutputCollector _collector; Object _id; Map _counts = new HashMap(); @Override public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { _collector = collector; _id = id; } @Override public void execute(Tuple tuple) { Object key = tuple.getValue(1); int curr = Utils.get(_counts, key, 0); _counts.put(key, curr + 1); } @Override public void finishBatch() { for(Object key: _counts.keySet()) { _collector.emit(new Values(_id, key, _counts.get(key))); } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("tx", "key", "count")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/KeyedCountingCommitterBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.transactional.ICommitter; public class KeyedCountingCommitterBolt extends KeyedCountingBatchBolt implements ICommitter { } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/KeyedSummingBatchBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBatchBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import backtype.storm.utils.Utils; import clojure.lang.Numbers; import java.util.HashMap; import java.util.Map; public class KeyedSummingBatchBolt extends BaseBatchBolt { BatchOutputCollector _collector; Object _id; Map _sums = new HashMap(); @Override public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { _collector = collector; _id = id; } @Override public void execute(Tuple tuple) { Object key = tuple.getValue(1); Number curr = Utils.get(_sums, key, 0); _sums.put(key, Numbers.add(curr, tuple.getValue(2))); } @Override public void finishBatch() { for(Object key: _sums.keySet()) { _collector.emit(new Values(_id, key, _sums.get(key))); } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("tx", "key", "sum")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/MemoryTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.Config; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.transactional.TransactionAttempt; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout.Emitter; import backtype.storm.tuple.Fields; import backtype.storm.utils.RegisteredGlobalState; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; public class MemoryTransactionalSpout implements IPartitionedTransactionalSpout { public static String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id"; private String _id; private String _finishedPartitionsId; private int _takeAmt; private Fields _outFields; private Map>> _initialPartitions; public MemoryTransactionalSpout(Map>> partitions, Fields outFields, int takeAmt) { _id = RegisteredGlobalState.registerState(partitions); Map finished = Collections.synchronizedMap(new HashMap()); _finishedPartitionsId = RegisteredGlobalState.registerState(finished); _takeAmt = takeAmt; _outFields = outFields; _initialPartitions = partitions; } public boolean isExhaustedTuples() { Map statuses = getFinishedStatuses(); for(Integer partition: getQueues().keySet()) { if(!statuses.containsKey(partition) || !getFinishedStatuses().get(partition)) { return false; } } return true; } class Coordinator implements IPartitionedTransactionalSpout.Coordinator { @Override public int numPartitions() { return getQueues().size(); } @Override public boolean isReady() { return true; } @Override public void close() { } } class Emitter implements IPartitionedTransactionalSpout.Emitter { Integer _maxSpoutPending; Map _emptyPartitions = new HashMap(); public Emitter(Map conf) { Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); if(c==null) _maxSpoutPending = 1; else _maxSpoutPending = Utils.getInt(c); } @Override public MemoryTransactionalSpoutMeta emitPartitionBatchNew(TransactionAttempt tx, BatchOutputCollector collector, int partition, MemoryTransactionalSpoutMeta lastPartitionMeta) { int index; if(lastPartitionMeta==null) { index = 0; } else { index = lastPartitionMeta.index + lastPartitionMeta.amt; } List> queue = getQueues().get(partition); int total = queue.size(); int left = total - index; int toTake = Math.min(left, _takeAmt); MemoryTransactionalSpoutMeta ret = new MemoryTransactionalSpoutMeta(index, toTake); emitPartitionBatch(tx, collector, partition, ret); if(toTake==0) { // this is a pretty hacky way to determine when all the partitions have been committed // wait until we've emitted max-spout-pending empty partitions for the partition int curr = Utils.get(_emptyPartitions, partition, 0) + 1; _emptyPartitions.put(partition, curr); if(curr > _maxSpoutPending) { Map finishedStatuses = getFinishedStatuses(); // will be null in remote mode if(finishedStatuses!=null) { finishedStatuses.put(partition, true); } } } return ret; } @Override public void emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, MemoryTransactionalSpoutMeta partitionMeta) { List> queue = getQueues().get(partition); for(int i=partitionMeta.index; i < partitionMeta.index + partitionMeta.amt; i++) { List toEmit = new ArrayList(queue.get(i)); toEmit.add(0, tx); collector.emit(toEmit); } } @Override public void close() { } } @Override public IPartitionedTransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { return new Coordinator(); } @Override public IPartitionedTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { return new Emitter(conf); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { List toDeclare = new ArrayList(_outFields.toList()); toDeclare.add(0, TX_FIELD); declarer.declare(new Fields(toDeclare)); } @Override public Map getComponentConfiguration() { Config conf = new Config(); conf.registerSerialization(MemoryTransactionalSpoutMeta.class); return conf; } public void startup() { getFinishedStatuses().clear(); } public void cleanup() { RegisteredGlobalState.clearState(_id); RegisteredGlobalState.clearState(_finishedPartitionsId); } private Map>> getQueues() { Map>> ret = (Map>>) RegisteredGlobalState.getState(_id); if(ret!=null) return ret; else return _initialPartitions; } private Map getFinishedStatuses() { return (Map) RegisteredGlobalState.getState(_finishedPartitionsId); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/MemoryTransactionalSpoutMeta.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; public class MemoryTransactionalSpoutMeta { int index; int amt; // for kryo compatibility public MemoryTransactionalSpoutMeta() { } public MemoryTransactionalSpoutMeta(int index, int amt) { this.index = index; this.amt = amt; } @Override public String toString() { return "index: " + index + "; amt: " + amt; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/MkClusterParam.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import java.util.Map; /** * The param arg for Testing.withSimulatedTimeCluster and Testing.withTrackedCluster */ public class MkClusterParam { /** * count of supervisors for the cluster. */ private Integer supervisors; /** * count of port for each supervisor */ private Integer portsPerSupervisor; /** * cluster config */ private Map daemonConf; public Integer getSupervisors() { return supervisors; } public void setSupervisors(Integer supervisors) { this.supervisors = supervisors; } public Integer getPortsPerSupervisor() { return portsPerSupervisor; } public void setPortsPerSupervisor(Integer portsPerSupervisor) { this.portsPerSupervisor = portsPerSupervisor; } public Map getDaemonConf() { return daemonConf; } public void setDaemonConf(Map daemonConf) { this.daemonConf = daemonConf; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/MkTupleParam.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import java.util.ArrayList; import java.util.List; public class MkTupleParam { private String stream; private String component; private List fields; public String getStream() { return stream; } public void setStream(String stream) { this.stream = stream; } public String getComponent() { return component; } public void setComponent(String component) { this.component = component; } public List getFields() { return fields; } public void setFields(String... fields) { this.fields = new ArrayList(); for (int i = 0; i < fields.length; i++) { this.fields.add(fields[i]); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/MockedSources.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import backtype.storm.tuple.Values; import backtype.storm.utils.Utils; public class MockedSources { /** * mocked spout sources for the [spout, stream] pair. */ private Map> data = new HashMap>(); /** * add mock data for the spout. * * @param spoutId the spout to be mocked * @param streamId the stream of the spout to be mocked * @param objects the mocked data */ public void addMockData(String spoutId, String streamId, Values... valueses) { if (!data.containsKey(spoutId)) { data.put(spoutId, new ArrayList()); } List tuples = data.get(spoutId); for (int i = 0; i < valueses.length; i++) { FixedTuple tuple = new FixedTuple(streamId, valueses[i]); tuples.add(tuple); } } public void addMockData(String spoutId, Values... valueses) { this.addMockData(spoutId, Utils.DEFAULT_STREAM_ID, valueses); } public Map> getData() { return this.data; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/NGrouping.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.generated.GlobalStreamId; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.task.WorkerTopologyContext; import java.util.ArrayList; import java.util.Collections; import java.util.List; public class NGrouping implements CustomStreamGrouping { int _n; List _outTasks; public NGrouping(int n) { _n = n; } @Override public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) { targetTasks = new ArrayList(targetTasks); Collections.sort(targetTasks); _outTasks = new ArrayList(); for(int i=0; i<_n; i++) { _outTasks.add(targetTasks.get(i)); } } @Override public List chooseTasks(int taskId, List values) { return _outTasks; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/NonRichBoltTracker.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.task.IBolt; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Tuple; import backtype.storm.utils.RegisteredGlobalState; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; public class NonRichBoltTracker implements IBolt { IBolt _delegate; String _trackId; public NonRichBoltTracker(IBolt delegate, String id) { _delegate = delegate; _trackId = id; } public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { _delegate.prepare(stormConf, context, collector); } public void execute(Tuple input) { _delegate.execute(input); Map stats = (Map) RegisteredGlobalState.getState(_trackId); ((AtomicInteger) stats.get("processed")).incrementAndGet(); } public void cleanup() { _delegate.cleanup(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.Config; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.transactional.TransactionAttempt; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; import backtype.storm.tuple.Fields; import backtype.storm.utils.RegisteredGlobalState; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; /** * This spout only works in local mode. */ public class OpaqueMemoryTransactionalSpout implements IOpaquePartitionedTransactionalSpout { public static String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id"; private String _id; private String _finishedPartitionsId; private String _disabledId; private int _takeAmt; private Fields _outFields; public OpaqueMemoryTransactionalSpout(Map>> partitions, Fields outFields, int takeAmt) { _id = RegisteredGlobalState.registerState(partitions); Map finished = Collections.synchronizedMap(new HashMap()); _finishedPartitionsId = RegisteredGlobalState.registerState(finished); Map disabled = Collections.synchronizedMap(new HashMap()); _disabledId = RegisteredGlobalState.registerState(disabled); _takeAmt = takeAmt; _outFields = outFields; } public void setDisabled(Integer partition, boolean disabled) { getDisabledStatuses().put(partition, disabled); } public boolean isExhaustedTuples() { Map statuses = getFinishedStatuses(); for(Integer partition: getQueues().keySet()) { if(!statuses.containsKey(partition) || !getFinishedStatuses().get(partition)) { return false; } } return true; } @Override public IOpaquePartitionedTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { return new Emitter(conf); } @Override public IOpaquePartitionedTransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { return new Coordinator(); } class Coordinator implements IOpaquePartitionedTransactionalSpout.Coordinator { @Override public boolean isReady() { return true; } @Override public void close() { } } class Emitter implements IOpaquePartitionedTransactionalSpout.Emitter { Integer _maxSpoutPending; Map _emptyPartitions = new HashMap(); public Emitter(Map conf) { Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); if(c==null) _maxSpoutPending = 1; else _maxSpoutPending = Utils.getInt(c); } @Override public MemoryTransactionalSpoutMeta emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, MemoryTransactionalSpoutMeta lastPartitionMeta) { if(!Boolean.FALSE.equals(getDisabledStatuses().get(partition))) { int index; if(lastPartitionMeta==null) { index = 0; } else { index = lastPartitionMeta.index + lastPartitionMeta.amt; } List> queue = getQueues().get(partition); int total = queue.size(); int left = total - index; int toTake = Math.min(left, _takeAmt); MemoryTransactionalSpoutMeta ret = new MemoryTransactionalSpoutMeta(index, toTake); for(int i=ret.index; i < ret.index + ret.amt; i++) { List toEmit = new ArrayList(queue.get(i)); toEmit.add(0, tx); collector.emit(toEmit); } if(toTake==0) { // this is a pretty hacky way to determine when all the partitions have been committed // wait until we've emitted max-spout-pending empty partitions for the partition int curr = Utils.get(_emptyPartitions, partition, 0) + 1; _emptyPartitions.put(partition, curr); if(curr > _maxSpoutPending) { getFinishedStatuses().put(partition, true); } } return ret; } else { return null; } } @Override public void close() { } @Override public int numPartitions() { return getQueues().size(); } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { List toDeclare = new ArrayList(_outFields.toList()); toDeclare.add(0, TX_FIELD); declarer.declare(new Fields(toDeclare)); } @Override public Map getComponentConfiguration() { Config conf = new Config(); conf.registerSerialization(MemoryTransactionalSpoutMeta.class); return conf; } public void startup() { getFinishedStatuses().clear(); } public void cleanup() { RegisteredGlobalState.clearState(_id); RegisteredGlobalState.clearState(_finishedPartitionsId); } private Map>> getQueues() { return (Map>>) RegisteredGlobalState.getState(_id); } private Map getFinishedStatuses() { return (Map) RegisteredGlobalState.getState(_finishedPartitionsId); } private Map getDisabledStatuses() { return (Map) RegisteredGlobalState.getState(_disabledId); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/PrepareBatchBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.BasicOutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBasicBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.List; public class PrepareBatchBolt extends BaseBasicBolt { Fields _outFields; public PrepareBatchBolt(Fields outFields) { _outFields = outFields; } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(_outFields); } @Override public void execute(Tuple input, BasicOutputCollector collector) { long id = Utils.secureRandomLong(); List toEmit = new ArrayList(); toEmit.add(id); toEmit.addAll(input.getValues()); collector.emit(toEmit); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/SpoutTracker.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.spout.ISpoutOutputCollector; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichSpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.utils.RegisteredGlobalState; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; public class SpoutTracker extends BaseRichSpout { IRichSpout _delegate; SpoutTrackOutputCollector _tracker; String _trackId; private class SpoutTrackOutputCollector implements ISpoutOutputCollector { public int transferred = 0; public int emitted = 0; public SpoutOutputCollector _collector; public SpoutTrackOutputCollector(SpoutOutputCollector collector) { _collector = collector; } private void recordSpoutEmit() { Map stats = (Map) RegisteredGlobalState.getState(_trackId); ((AtomicInteger) stats.get("spout-emitted")).incrementAndGet(); } public List emit(String streamId, List tuple, Object messageId) { List ret = _collector.emit(streamId, tuple, messageId); recordSpoutEmit(); return ret; } public void emitDirect(int taskId, String streamId, List tuple, Object messageId) { _collector.emitDirect(taskId, streamId, tuple, messageId); recordSpoutEmit(); } @Override public void reportError(Throwable error) { _collector.reportError(error); } } public SpoutTracker(IRichSpout delegate, String trackId) { _delegate = delegate; _trackId = trackId; } public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _tracker = new SpoutTrackOutputCollector(collector); _delegate.open(conf, context, new SpoutOutputCollector(_tracker)); } public void close() { _delegate.close(); } public void nextTuple() { _delegate.nextTuple(); } public void ack(Object msgId) { _delegate.ack(msgId); Map stats = (Map) RegisteredGlobalState.getState(_trackId); ((AtomicInteger) stats.get("processed")).incrementAndGet(); } public void fail(Object msgId) { _delegate.fail(msgId); Map stats = (Map) RegisteredGlobalState.getState(_trackId); ((AtomicInteger) stats.get("processed")).incrementAndGet(); } public void declareOutputFields(OutputFieldsDeclarer declarer) { _delegate.declareOutputFields(declarer); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestAggregatesCounter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.base.BaseRichBolt; import backtype.storm.task.OutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Fields; import java.util.Map; import backtype.storm.task.TopologyContext; import java.util.HashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static backtype.storm.utils.Utils.tuple; public class TestAggregatesCounter extends BaseRichBolt { public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); Map _counts; OutputCollector _collector; public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { _collector = collector; _counts = new HashMap(); } public void execute(Tuple input) { String word = (String) input.getValues().get(0); int count = (Integer) input.getValues().get(1); _counts.put(word, count); int globalCount = 0; for(String w: _counts.keySet()) { globalCount+=_counts.get(w); } _collector.emit(tuple(globalCount)); _collector.ack(input); } public void cleanup() { } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("agg-global")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestConfBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.task.TopologyContext; import backtype.storm.topology.BasicOutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseBasicBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import java.util.Map; public class TestConfBolt extends BaseBasicBolt { Map _componentConf; Map _conf; public TestConfBolt() { this(null); } public TestConfBolt(Map componentConf) { _componentConf = componentConf; } @Override public void prepare(Map conf, TopologyContext context) { _conf = conf; } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("conf", "value")); } @Override public void execute(Tuple input, BasicOutputCollector collector) { String name = input.getString(0); collector.emit(new Values(name, _conf.get(name))); } @Override public Map getComponentConfiguration() { return _componentConf; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestGlobalCount.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.base.BaseRichBolt; import backtype.storm.task.OutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Fields; import java.util.Map; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Values; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TestGlobalCount extends BaseRichBolt { public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); private int _count; OutputCollector _collector; public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { _collector = collector; _count = 0; } public void execute(Tuple input) { _count++; _collector.emit(input, new Values(_count)); _collector.ack(input); } public void cleanup() { } public Fields getOutputFields() { return new Fields("global-count"); } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("global-count")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestJob.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.ILocalCluster; /** * This is the core interface for the storm java testing, usually * we put our java unit testing logic in the run method. A sample * code will be: * * Testing.withSimulatedTimeLocalCluster(new TestJob() { * public void run(Cluster cluster) { * // your testing logic here. * } * }); */ public interface TestJob { /** * run the testing logic with the cluster. * * @param cluster the cluster which created by Testing.withSimulatedTimeLocalCluster * and Testing.withTrackedCluster. */ public void run(ILocalCluster cluster) throws Exception; } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestKryoDecorator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.serialization.IKryoDecorator; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; public class TestKryoDecorator implements IKryoDecorator { public void decorate(Kryo k) { k.register(TestSerObject.class); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestPlannerBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.task.OutputCollector; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Fields; import java.util.Map; import backtype.storm.task.TopologyContext; import backtype.storm.topology.base.BaseRichBolt; public class TestPlannerBolt extends BaseRichBolt { public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { } public void execute(Tuple input) { } public Fields getOutputFields() { return new Fields("field1", "field2"); } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(getOutputFields()); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestPlannerSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.Config; import backtype.storm.topology.OutputFieldsDeclarer; import java.util.Map; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import java.util.HashMap; public class TestPlannerSpout extends BaseRichSpout { boolean _isDistributed; Fields _outFields; public TestPlannerSpout(Fields outFields, boolean isDistributed) { _isDistributed = isDistributed; _outFields = outFields; } public TestPlannerSpout(boolean isDistributed) { this(new Fields("field1", "field2"), isDistributed); } public TestPlannerSpout(Fields outFields) { this(outFields, true); } public Fields getOutputFields() { return _outFields; } public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { } public void close() { } public void nextTuple() { Utils.sleep(100); } public void ack(Object msgId){ } public void fail(Object msgId){ } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(getOutputFields()); } @Override public Map getComponentConfiguration() { Map ret = new HashMap(); if(!_isDistributed) { ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1); } return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestSerObject.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import java.io.Serializable; public class TestSerObject implements Serializable { public int f1; public int f2; public TestSerObject(int f1, int f2) { this.f1 = f1; this.f2 = f2; } @Override public boolean equals(Object o) { TestSerObject other = (TestSerObject) o; return f1 == other.f1 && f2 == other.f2; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestWordCounter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.topology.base.BaseBasicBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Fields; import java.util.Map; import backtype.storm.task.TopologyContext; import backtype.storm.topology.BasicOutputCollector; import java.util.HashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static backtype.storm.utils.Utils.tuple; public class TestWordCounter extends BaseBasicBolt { public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); Map _counts; public void prepare(Map stormConf, TopologyContext context) { _counts = new HashMap(); } public void execute(Tuple input, BasicOutputCollector collector) { String word = (String) input.getValues().get(0); int count = 0; if(_counts.containsKey(word)) { count = _counts.get(word); } count++; _counts.put(word, count); collector.emit(tuple(word, count)); } public void cleanup() { } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("word", "count")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TestWordSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.Config; import backtype.storm.topology.OutputFieldsDeclarer; import java.util.Map; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import backtype.storm.utils.Utils; import java.util.HashMap; import java.util.Random; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TestWordSpout extends BaseRichSpout { public static Logger LOG = LoggerFactory.getLogger(TestWordSpout.class); boolean _isDistributed; SpoutOutputCollector _collector; public TestWordSpout() { this(true); } public TestWordSpout(boolean isDistributed) { _isDistributed = isDistributed; } public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _collector = collector; } public void close() { } public void nextTuple() { Utils.sleep(100); final String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"}; final Random rand = new Random(); final String word = words[rand.nextInt(words.length)]; _collector.emit(new Values(word)); } public void ack(Object msgId) { } public void fail(Object msgId) { } public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("word")); } @Override public Map getComponentConfiguration() { if(!_isDistributed) { Map ret = new HashMap(); ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1); return ret; } else { return null; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TrackedTopology.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import java.util.HashMap; import java.util.Map; import backtype.storm.generated.StormTopology; import clojure.lang.Keyword; public class TrackedTopology extends HashMap{ public TrackedTopology(Map map) { super(map); } public StormTopology getTopology() { return (StormTopology)get(Keyword.intern("topology")); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/testing/TupleCaptureBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.testing; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; public class TupleCaptureBolt implements IRichBolt { public static transient Map>> emitted_tuples = new HashMap>>(); private String _name; private OutputCollector _collector; public TupleCaptureBolt() { _name = UUID.randomUUID().toString(); emitted_tuples.put(_name, new HashMap>()); } public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { _collector = collector; } public void execute(Tuple input) { String component = input.getSourceComponent(); Map> captured = emitted_tuples.get(_name); if(!captured.containsKey(component)) { captured.put(component, new ArrayList()); } captured.get(component).add(new FixedTuple(input.getSourceStreamId(), input.getValues())); _collector.ack(input); } public Map> getResults() { return emitted_tuples.get(_name); } public void cleanup() { } public Map> getAndRemoveResults() { return emitted_tuples.remove(_name); } public Map> getAndClearResults() { Map> ret = new HashMap>(emitted_tuples.get(_name)); emitted_tuples.get(_name).clear(); return ret; } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { } @Override public Map getComponentConfiguration() { return null; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/BaseConfigurationDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.Config; import java.util.HashMap; import java.util.Map; public abstract class BaseConfigurationDeclarer implements ComponentConfigurationDeclarer { @Override public T addConfiguration(String config, Object value) { Map configMap = new HashMap(); configMap.put(config, value); return addConfigurations(configMap); } @Override public T setDebug(boolean debug) { return addConfiguration(Config.TOPOLOGY_DEBUG, debug); } @Override public T setMaxTaskParallelism(Number val) { if(val!=null) val = val.intValue(); return addConfiguration(Config.TOPOLOGY_MAX_TASK_PARALLELISM, val); } @Override public T setMaxSpoutPending(Number val) { if(val!=null) val = val.intValue(); return addConfiguration(Config.TOPOLOGY_MAX_SPOUT_PENDING, val); } @Override public T setNumTasks(Number val) { if(val!=null) val = val.intValue(); return addConfiguration(Config.TOPOLOGY_TASKS, val); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/BasicBoltExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Tuple; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class BasicBoltExecutor implements IRichBolt { public static Logger LOG = LoggerFactory.getLogger(BasicBoltExecutor.class); private IBasicBolt _bolt; private transient BasicOutputCollector _collector; public BasicBoltExecutor(IBasicBolt bolt) { _bolt = bolt; } public void declareOutputFields(OutputFieldsDeclarer declarer) { _bolt.declareOutputFields(declarer); } public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { _bolt.prepare(stormConf, context); _collector = new BasicOutputCollector(collector); } public void execute(Tuple input) { _collector.setContext(input); try { _bolt.execute(input, _collector); _collector.getOutputter().ack(input); } catch(FailedException e) { if(e instanceof ReportedFailedException) { _collector.reportError(e); } _collector.getOutputter().fail(input); } } public void cleanup() { _bolt.cleanup(); } public Map getComponentConfiguration() { return _bolt.getComponentConfiguration(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/BasicOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.task.IOutputCollector; import backtype.storm.task.OutputCollector; import backtype.storm.tuple.Tuple; import backtype.storm.utils.Utils; import java.util.List; public class BasicOutputCollector implements IBasicOutputCollector { private OutputCollector out; private Tuple inputTuple; public BasicOutputCollector(OutputCollector out) { this.out = out; } public List emit(String streamId, List tuple) { return out.emit(streamId, inputTuple, tuple); } public List emit(List tuple) { return emit(Utils.DEFAULT_STREAM_ID, tuple); } public void setContext(Tuple inputTuple) { this.inputTuple = inputTuple; } public void emitDirect(int taskId, String streamId, List tuple) { out.emitDirect(taskId, streamId, inputTuple, tuple); } public void emitDirect(int taskId, List tuple) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); } protected IOutputCollector getOutputter() { return out; } public void reportError(Throwable t) { out.reportError(t); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/BoltDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; public interface BoltDeclarer extends InputDeclarer, ComponentConfigurationDeclarer { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/ComponentConfigurationDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import java.util.Map; public interface ComponentConfigurationDeclarer { T addConfigurations(Map conf); T addConfiguration(String config, Object value); T setDebug(boolean debug); T setMaxTaskParallelism(Number val); T setMaxSpoutPending(Number val); T setNumTasks(Number val); } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/FailedException.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; public class FailedException extends RuntimeException { public FailedException() { super(); } public FailedException(String msg) { super(msg); } public FailedException(String msg, Throwable cause) { super(msg, cause); } public FailedException(Throwable cause) { super(cause); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/IBasicBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Tuple; import java.util.Map; public interface IBasicBolt extends IComponent { void prepare(Map stormConf, TopologyContext context); /** * Process the input tuple and optionally emit new tuples based on the input tuple. * * All acking is managed for you. Throw a FailedException if you want to fail the tuple. */ void execute(Tuple input, BasicOutputCollector collector); void cleanup(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/IBasicOutputCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import java.util.List; public interface IBasicOutputCollector { List emit(String streamId, List tuple); void emitDirect(int taskId, String streamId, List tuple); void reportError(Throwable t); } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/IComponent.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import java.io.Serializable; import java.util.Map; /** * Common methods for all possible components in a topology. This interface is used * when defining topologies using the Java API. */ public interface IComponent extends Serializable { /** * Declare the output schema for all the streams of this topology. * * @param declarer this is used to declare output stream ids, output fields, and whether or not each output stream is a direct stream */ void declareOutputFields(OutputFieldsDeclarer declarer); /** * Declare configuration specific to this component. Only a subset of the "topology.*" configs can * be overridden. The component configuration can be further overridden when constructing the * topology using {@link TopologyBuilder} * */ Map getComponentConfiguration(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/IRichBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.task.IBolt; /** * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces * to use to implement components of the topology. * */ public interface IRichBolt extends IBolt, IComponent { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/IRichSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.spout.ISpout; /** * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces * to use to implement components of the topology. * */ public interface IRichSpout extends ISpout, IComponent { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/IRichStateSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.state.IStateSpout; public interface IRichStateSpout extends IStateSpout, IComponent { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/InputDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.tuple.Fields; public interface InputDeclarer { public T fieldsGrouping(String componentId, Fields fields); public T fieldsGrouping(String componentId, String streamId, Fields fields); public T globalGrouping(String componentId); public T globalGrouping(String componentId, String streamId); public T shuffleGrouping(String componentId); public T shuffleGrouping(String componentId, String streamId); public T localOrShuffleGrouping(String componentId); public T localOrShuffleGrouping(String componentId, String streamId); public T noneGrouping(String componentId); public T noneGrouping(String componentId, String streamId); public T allGrouping(String componentId); public T allGrouping(String componentId, String streamId); public T directGrouping(String componentId); public T directGrouping(String componentId, String streamId); public T customGrouping(String componentId, CustomStreamGrouping grouping); public T customGrouping(String componentId, String streamId, CustomStreamGrouping grouping); public T grouping(GlobalStreamId id, Grouping grouping); } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/OutputFieldsDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.tuple.Fields; public interface OutputFieldsDeclarer { /** * Uses default stream id. */ public void declare(Fields fields); public void declare(boolean direct, Fields fields); public void declareStream(String streamId, Fields fields); public void declareStream(String streamId, boolean direct, Fields fields); } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/OutputFieldsGetter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.generated.StreamInfo; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import java.util.HashMap; import java.util.Map; public class OutputFieldsGetter implements OutputFieldsDeclarer { private Map _fields = new HashMap(); public void declare(Fields fields) { declare(false, fields); } public void declare(boolean direct, Fields fields) { declareStream(Utils.DEFAULT_STREAM_ID, direct, fields); } public void declareStream(String streamId, Fields fields) { declareStream(streamId, false, fields); } public void declareStream(String streamId, boolean direct, Fields fields) { if(_fields.containsKey(streamId)) { throw new IllegalArgumentException("Fields for " + streamId + " already set"); } _fields.put(streamId, new StreamInfo(fields.toList(), direct)); } public Map getFieldsDeclaration() { return _fields; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/ReportedFailedException.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; public class ReportedFailedException extends FailedException { public ReportedFailedException() { super(); } public ReportedFailedException(String msg) { super(msg); } public ReportedFailedException(String msg, Throwable cause) { super(msg, cause); } public ReportedFailedException(Throwable cause) { super(cause); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/SpoutDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; public interface SpoutDeclarer extends ComponentConfigurationDeclarer { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology; import backtype.storm.Config; import backtype.storm.generated.Bolt; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.ComponentObject; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.generated.NullStruct; import backtype.storm.generated.SpoutSpec; import backtype.storm.generated.StateSpoutSpec; import backtype.storm.generated.StormTopology; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; import org.json.simple.JSONValue; /** * TopologyBuilder exposes the Java API for specifying a topology for Storm * to execute. Topologies are Thrift structures in the end, but since the Thrift API * is so verbose, TopologyBuilder greatly eases the process of creating topologies. * The template for creating and submitting a topology looks something like: * *
 * TopologyBuilder builder = new TopologyBuilder();
 *
 * builder.setSpout("1", new TestWordSpout(true), 5);
 * builder.setSpout("2", new TestWordSpout(true), 3);
 * builder.setBolt("3", new TestWordCounter(), 3)
 *          .fieldsGrouping("1", new Fields("word"))
 *          .fieldsGrouping("2", new Fields("word"));
 * builder.setBolt("4", new TestGlobalCount())
 *          .globalGrouping("1");
 *
 * Map conf = new HashMap();
 * conf.put(Config.TOPOLOGY_WORKERS, 4);
 * 
 * StormSubmitter.submitTopology("mytopology", conf, builder.createTopology());
 * 
* * Running the exact same topology in local mode (in process), and configuring it to log all tuples * emitted, looks like the following. Note that it lets the topology run for 10 seconds * before shutting down the local cluster. * *
 * TopologyBuilder builder = new TopologyBuilder();
 *
 * builder.setSpout("1", new TestWordSpout(true), 5);
 * builder.setSpout("2", new TestWordSpout(true), 3);
 * builder.setBolt("3", new TestWordCounter(), 3)
 *          .fieldsGrouping("1", new Fields("word"))
 *          .fieldsGrouping("2", new Fields("word"));
 * builder.setBolt("4", new TestGlobalCount())
 *          .globalGrouping("1");
 *
 * Map conf = new HashMap();
 * conf.put(Config.TOPOLOGY_WORKERS, 4);
 * conf.put(Config.TOPOLOGY_DEBUG, true);
 *
 * LocalCluster cluster = new LocalCluster();
 * cluster.submitTopology("mytopology", conf, builder.createTopology());
 * Utils.sleep(10000);
 * cluster.shutdown();
 * 
* *

The pattern for TopologyBuilder is to map component ids to components using the setSpout * and setBolt methods. Those methods return objects that are then used to declare * the inputs for that component.

*/ public class TopologyBuilder { private Map _bolts = new HashMap(); private Map _spouts = new HashMap(); private Map _commons = new HashMap(); // private Map> _inputs = new HashMap>(); private Map _stateSpouts = new HashMap(); public StormTopology createTopology() { Map boltSpecs = new HashMap(); Map spoutSpecs = new HashMap(); for(String boltId: _bolts.keySet()) { IRichBolt bolt = _bolts.get(boltId); ComponentCommon common = getComponentCommon(boltId, bolt); boltSpecs.put(boltId, new Bolt(ComponentObject.serialized_java(Utils.serialize(bolt)), common)); } for(String spoutId: _spouts.keySet()) { IRichSpout spout = _spouts.get(spoutId); ComponentCommon common = getComponentCommon(spoutId, spout); spoutSpecs.put(spoutId, new SpoutSpec(ComponentObject.serialized_java(Utils.serialize(spout)), common)); } return new StormTopology(spoutSpecs, boltSpecs, new HashMap()); } /** * Define a new bolt in this topology with parallelism of just one thread. * * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. * @param bolt the bolt * @return use the returned object to declare the inputs to this component */ public BoltDeclarer setBolt(String id, IRichBolt bolt) { return setBolt(id, bolt, null); } /** * Define a new bolt in this topology with the specified amount of parallelism. * * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. * @param bolt the bolt * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somewhere around the cluster. * @return use the returned object to declare the inputs to this component */ public BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism_hint) { validateUnusedId(id); initCommon(id, bolt, parallelism_hint); _bolts.put(id, bolt); return new BoltGetter(id); } /** * Define a new bolt in this topology. This defines a basic bolt, which is a * simpler to use but more restricted kind of bolt. Basic bolts are intended * for non-aggregation processing and automate the anchoring/acking process to * achieve proper reliability in the topology. * * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. * @param bolt the basic bolt * @return use the returned object to declare the inputs to this component */ public BoltDeclarer setBolt(String id, IBasicBolt bolt) { return setBolt(id, bolt, null); } /** * Define a new bolt in this topology. This defines a basic bolt, which is a * simpler to use but more restricted kind of bolt. Basic bolts are intended * for non-aggregation processing and automate the anchoring/acking process to * achieve proper reliability in the topology. * * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. * @param bolt the basic bolt * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somwehere around the cluster. * @return use the returned object to declare the inputs to this component */ public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism_hint) { return setBolt(id, new BasicBoltExecutor(bolt), parallelism_hint); } /** * Define a new spout in this topology. * * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs. * @param spout the spout */ public SpoutDeclarer setSpout(String id, IRichSpout spout) { return setSpout(id, spout, null); } /** * Define a new spout in this topology with the specified parallelism. If the spout declares * itself as non-distributed, the parallelism_hint will be ignored and only one task * will be allocated to this component. * * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs. * @param parallelism_hint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a process somwehere around the cluster. * @param spout the spout */ public SpoutDeclarer setSpout(String id, IRichSpout spout, Number parallelism_hint) { validateUnusedId(id); initCommon(id, spout, parallelism_hint); _spouts.put(id, spout); return new SpoutGetter(id); } public void setStateSpout(String id, IRichStateSpout stateSpout) { setStateSpout(id, stateSpout, null); } public void setStateSpout(String id, IRichStateSpout stateSpout, Number parallelism_hint) { validateUnusedId(id); // TODO: finish } private void validateUnusedId(String id) { if(_bolts.containsKey(id)) { throw new IllegalArgumentException("Bolt has already been declared for id " + id); } if(_spouts.containsKey(id)) { throw new IllegalArgumentException("Spout has already been declared for id " + id); } if(_stateSpouts.containsKey(id)) { throw new IllegalArgumentException("State spout has already been declared for id " + id); } } private ComponentCommon getComponentCommon(String id, IComponent component) { ComponentCommon ret = new ComponentCommon(_commons.get(id)); OutputFieldsGetter getter = new OutputFieldsGetter(); component.declareOutputFields(getter); ret.set_streams(getter.getFieldsDeclaration()); return ret; } private void initCommon(String id, IComponent component, Number parallelism) { ComponentCommon common = new ComponentCommon(); common.set_inputs(new HashMap()); if(parallelism!=null) common.set_parallelism_hint(parallelism.intValue()); Map conf = component.getComponentConfiguration(); if(conf!=null) common.set_json_conf(JSONValue.toJSONString(conf)); _commons.put(id, common); } protected class ConfigGetter extends BaseConfigurationDeclarer { String _id; public ConfigGetter(String id) { _id = id; } @Override public T addConfigurations(Map conf) { if(conf!=null && conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) { throw new IllegalArgumentException("Cannot set serializations for a component using fluent API"); } String currConf = _commons.get(_id).get_json_conf(); _commons.get(_id).set_json_conf(mergeIntoJson(parseJson(currConf), conf)); return (T) this; } } protected class SpoutGetter extends ConfigGetter implements SpoutDeclarer { public SpoutGetter(String id) { super(id); } } protected class BoltGetter extends ConfigGetter implements BoltDeclarer { private String _boltId; public BoltGetter(String boltId) { super(boltId); _boltId = boltId; } public BoltDeclarer fieldsGrouping(String componentId, Fields fields) { return fieldsGrouping(componentId, Utils.DEFAULT_STREAM_ID, fields); } public BoltDeclarer fieldsGrouping(String componentId, String streamId, Fields fields) { return grouping(componentId, streamId, Grouping.fields(fields.toList())); } public BoltDeclarer globalGrouping(String componentId) { return globalGrouping(componentId, Utils.DEFAULT_STREAM_ID); } public BoltDeclarer globalGrouping(String componentId, String streamId) { return grouping(componentId, streamId, Grouping.fields(new ArrayList())); } public BoltDeclarer shuffleGrouping(String componentId) { return shuffleGrouping(componentId, Utils.DEFAULT_STREAM_ID); } public BoltDeclarer shuffleGrouping(String componentId, String streamId) { return grouping(componentId, streamId, Grouping.shuffle(new NullStruct())); } public BoltDeclarer localOrShuffleGrouping(String componentId) { return localOrShuffleGrouping(componentId, Utils.DEFAULT_STREAM_ID); } public BoltDeclarer localOrShuffleGrouping(String componentId, String streamId) { return grouping(componentId, streamId, Grouping.local_or_shuffle(new NullStruct())); } public BoltDeclarer noneGrouping(String componentId) { return noneGrouping(componentId, Utils.DEFAULT_STREAM_ID); } public BoltDeclarer noneGrouping(String componentId, String streamId) { return grouping(componentId, streamId, Grouping.none(new NullStruct())); } public BoltDeclarer allGrouping(String componentId) { return allGrouping(componentId, Utils.DEFAULT_STREAM_ID); } public BoltDeclarer allGrouping(String componentId, String streamId) { return grouping(componentId, streamId, Grouping.all(new NullStruct())); } public BoltDeclarer directGrouping(String componentId) { return directGrouping(componentId, Utils.DEFAULT_STREAM_ID); } public BoltDeclarer directGrouping(String componentId, String streamId) { return grouping(componentId, streamId, Grouping.direct(new NullStruct())); } private BoltDeclarer grouping(String componentId, String streamId, Grouping grouping) { _commons.get(_boltId).put_to_inputs(new GlobalStreamId(componentId, streamId), grouping); return this; } @Override public BoltDeclarer customGrouping(String componentId, CustomStreamGrouping grouping) { return customGrouping(componentId, Utils.DEFAULT_STREAM_ID, grouping); } @Override public BoltDeclarer customGrouping(String componentId, String streamId, CustomStreamGrouping grouping) { return grouping(componentId, streamId, Grouping.custom_serialized(Utils.serialize(grouping))); } @Override public BoltDeclarer grouping(GlobalStreamId id, Grouping grouping) { return grouping(id.get_componentId(), id.get_streamId(), grouping); } } private static Map parseJson(String json) { if(json==null) return new HashMap(); else return (Map) JSONValue.parse(json); } private static String mergeIntoJson(Map into, Map newMap) { Map res = new HashMap(into); if(newMap!=null) res.putAll(newMap); return JSONValue.toJSONString(res); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseBasicBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IBasicBolt; import java.util.Map; public abstract class BaseBasicBolt extends BaseComponent implements IBasicBolt { @Override public void prepare(Map stormConf, TopologyContext context) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseBatchBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.coordination.IBatchBolt; import java.util.Map; public abstract class BaseBatchBolt extends BaseComponent implements IBatchBolt { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseComponent.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.topology.IComponent; import java.util.Map; public abstract class BaseComponent implements IComponent { @Override public Map getComponentConfiguration() { return null; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; public abstract class BaseOpaquePartitionedTransactionalSpout extends BaseComponent implements IOpaquePartitionedTransactionalSpout { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; import java.util.Map; public abstract class BasePartitionedTransactionalSpout extends BaseComponent implements IPartitionedTransactionalSpout { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseRichBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.topology.IRichBolt; public abstract class BaseRichBolt extends BaseComponent implements IRichBolt { @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseRichSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /* * To change this template, choose Tools | Templates * and open the template in the editor. */ package backtype.storm.topology.base; import backtype.storm.topology.IRichSpout; /** * * @author nathan */ public abstract class BaseRichSpout extends BaseComponent implements IRichSpout { @Override public void close() { } @Override public void activate() { } @Override public void deactivate() { } @Override public void ack(Object msgId) { } @Override public void fail(Object msgId) { } } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.transactional.TransactionAttempt; public abstract class BaseTransactionalBolt extends BaseBatchBolt { } ================================================ FILE: storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.topology.base; import backtype.storm.transactional.ITransactionalSpout; import java.util.Map; public abstract class BaseTransactionalSpout extends BaseComponent implements ITransactionalSpout { } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/ICommitter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional; /** * This marks an IBatchBolt within a transactional topology as a committer. This causes the * finishBatch method to be called in order of the transactions. */ public interface ICommitter { } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/ICommitterTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional; import backtype.storm.task.TopologyContext; import java.util.Map; public interface ICommitterTransactionalSpout extends ITransactionalSpout { public interface Emitter extends ITransactionalSpout.Emitter { void commit(TransactionAttempt attempt); } @Override public Emitter getEmitter(Map conf, TopologyContext context); } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/ITransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IComponent; import java.math.BigInteger; import java.util.Map; public interface ITransactionalSpout extends IComponent { public interface Coordinator { /** * Create metadata for this particular transaction id which has never * been emitted before. The metadata should contain whatever is necessary * to be able to replay the exact batch for the transaction at a later point. * * The metadata is stored in Zookeeper. * * Storm uses the Kryo serializations configured in the component configuration * for this spout to serialize and deserialize the metadata. * * @param txid The id of the transaction. * @param prevMetadata The metadata of the previous transaction * @return the metadata for this new transaction */ X initializeTransaction(BigInteger txid, X prevMetadata); /** * Returns true if its ok to emit start a new transaction, false otherwise (will skip this transaction). * * You should sleep here if you want a delay between asking for the next transaction (this will be called * repeatedly in a loop). */ boolean isReady(); /** * Release any resources from this coordinator. */ void close(); } public interface Emitter { /** * Emit a batch for the specified transaction attempt and metadata for the transaction. The metadata * was created by the Coordinator in the initializeTranaction method. This method must always emit * the same batch of tuples across all tasks for the same transaction id. * * The first field of all emitted tuples must contain the provided TransactionAttempt. * */ void emitBatch(TransactionAttempt tx, X coordinatorMeta, BatchOutputCollector collector); /** * Any state for transactions prior to the provided transaction id can be safely cleaned up, so this * method should clean up that state. */ void cleanupBefore(BigInteger txid); /** * Release any resources held by this emitter. */ void close(); } /** * The coordinator for a TransactionalSpout runs in a single thread and indicates when batches * of tuples should be emitted and when transactions should commit. The Coordinator that you provide * in a TransactionalSpout provides metadata for each transaction so that the transactions can be replayed. */ Coordinator getCoordinator(Map conf, TopologyContext context); /** * The emitter for a TransactionalSpout runs as many tasks across the cluster. Emitters are responsible for * emitting batches of tuples for a transaction and must ensure that the same batch of tuples is always * emitted for the same transaction id. */ Emitter getEmitter(Map conf, TopologyContext context); } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/TransactionAttempt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional; import java.math.BigInteger; public class TransactionAttempt { BigInteger _txid; long _attemptId; // for kryo compatibility public TransactionAttempt() { } public TransactionAttempt(BigInteger txid, long attemptId) { _txid = txid; _attemptId = attemptId; } public BigInteger getTransactionId() { return _txid; } public long getAttemptId() { return _attemptId; } @Override public int hashCode() { return _txid.hashCode(); } @Override public boolean equals(Object o) { if(!(o instanceof TransactionAttempt)) return false; TransactionAttempt other = (TransactionAttempt) o; return _txid.equals(other._txid) && _attemptId == other._attemptId; } @Override public String toString() { return "" + _txid + ":" + _attemptId; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional; import backtype.storm.coordination.BatchOutputCollectorImpl; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.FailedException; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Tuple; import java.math.BigInteger; import java.util.Map; import java.util.TreeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TransactionalSpoutBatchExecutor implements IRichBolt { public static Logger LOG = LoggerFactory.getLogger(TransactionalSpoutBatchExecutor.class); BatchOutputCollectorImpl _collector; ITransactionalSpout _spout; ITransactionalSpout.Emitter _emitter; TreeMap _activeTransactions = new TreeMap(); public TransactionalSpoutBatchExecutor(ITransactionalSpout spout) { _spout = spout; } @Override public void prepare(Map conf, TopologyContext context, OutputCollector collector) { _collector = new BatchOutputCollectorImpl(collector); _emitter = _spout.getEmitter(conf, context); } @Override public void execute(Tuple input) { TransactionAttempt attempt = (TransactionAttempt) input.getValue(0); try { if(input.getSourceStreamId().equals(TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID)) { if(attempt.equals(_activeTransactions.get(attempt.getTransactionId()))) { ((ICommitterTransactionalSpout.Emitter) _emitter).commit(attempt); _activeTransactions.remove(attempt.getTransactionId()); _collector.ack(input); } else { _collector.fail(input); } } else { _emitter.emitBatch(attempt, input.getValue(1), _collector); _activeTransactions.put(attempt.getTransactionId(), attempt); _collector.ack(input); BigInteger committed = (BigInteger) input.getValue(2); if(committed!=null) { // valid to delete before what's been committed since // those batches will never be accessed again _activeTransactions.headMap(committed).clear(); _emitter.cleanupBefore(committed); } } } catch(FailedException e) { LOG.warn("Failed to emit batch for transaction", e); _collector.fail(input); } } @Override public void cleanup() { _emitter.close(); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { _spout.declareOutputFields(declarer); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutCoordinator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional; import backtype.storm.Config; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.FailedException; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.transactional.state.RotatingTransactionalState; import backtype.storm.transactional.state.TransactionalState; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import backtype.storm.utils.Utils; import java.math.BigInteger; import java.util.Map; import java.util.TreeMap; import java.util.Random; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TransactionalSpoutCoordinator extends BaseRichSpout { public static final Logger LOG = LoggerFactory.getLogger(TransactionalSpoutCoordinator.class); public static final BigInteger INIT_TXID = BigInteger.ONE; public static final String TRANSACTION_BATCH_STREAM_ID = TransactionalSpoutCoordinator.class.getName() + "/batch"; public static final String TRANSACTION_COMMIT_STREAM_ID = TransactionalSpoutCoordinator.class.getName() + "/commit"; private static final String CURRENT_TX = "currtx"; private static final String META_DIR = "meta"; private ITransactionalSpout _spout; private ITransactionalSpout.Coordinator _coordinator; private TransactionalState _state; private RotatingTransactionalState _coordinatorState; TreeMap _activeTx = new TreeMap(); private SpoutOutputCollector _collector; private Random _rand; BigInteger _currTransaction; int _maxTransactionActive; StateInitializer _initializer; public TransactionalSpoutCoordinator(ITransactionalSpout spout) { _spout = spout; } public ITransactionalSpout getSpout() { return _spout; } @Override public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _rand = new Random(Utils.secureRandomLong()); _state = TransactionalState.newCoordinatorState(conf, (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), _spout.getComponentConfiguration()); _coordinatorState = new RotatingTransactionalState(_state, META_DIR, true); _collector = collector; _coordinator = _spout.getCoordinator(conf, context); _currTransaction = getStoredCurrTransaction(_state); Object active = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); if(active==null) { _maxTransactionActive = 1; } else { _maxTransactionActive = Utils.getInt(active); } _initializer = new StateInitializer(); } @Override public void close() { _state.close(); } @Override public void nextTuple() { sync(); } @Override public void ack(Object msgId) { TransactionAttempt tx = (TransactionAttempt) msgId; TransactionStatus status = _activeTx.get(tx.getTransactionId()); if(status!=null && tx.equals(status.attempt)) { if(status.status==AttemptStatus.PROCESSING) { status.status = AttemptStatus.PROCESSED; } else if(status.status==AttemptStatus.COMMITTING) { _activeTx.remove(tx.getTransactionId()); _coordinatorState.cleanupBefore(tx.getTransactionId()); _currTransaction = nextTransactionId(tx.getTransactionId()); _state.setData(CURRENT_TX, _currTransaction); } sync(); } } @Override public void fail(Object msgId) { TransactionAttempt tx = (TransactionAttempt) msgId; TransactionStatus stored = _activeTx.remove(tx.getTransactionId()); if(stored!=null && tx.equals(stored.attempt)) { _activeTx.tailMap(tx.getTransactionId()).clear(); sync(); } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { // in partitioned example, in case an emitter task receives a later transaction than it's emitted so far, // when it sees the earlier txid it should know to emit nothing declarer.declareStream(TRANSACTION_BATCH_STREAM_ID, new Fields("tx", "tx-meta", "committed-txid")); declarer.declareStream(TRANSACTION_COMMIT_STREAM_ID, new Fields("tx")); } private void sync() { // note that sometimes the tuples active may be less than max_spout_pending, e.g. // max_spout_pending = 3 // tx 1, 2, 3 active, tx 2 is acked. there won't be a commit for tx 2 (because tx 1 isn't committed yet), // and there won't be a batch for tx 4 because there's max_spout_pending tx active TransactionStatus maybeCommit = _activeTx.get(_currTransaction); if(maybeCommit!=null && maybeCommit.status == AttemptStatus.PROCESSED) { maybeCommit.status = AttemptStatus.COMMITTING; _collector.emit(TRANSACTION_COMMIT_STREAM_ID, new Values(maybeCommit.attempt), maybeCommit.attempt); } try { if(_activeTx.size() < _maxTransactionActive) { BigInteger curr = _currTransaction; for(int i=0; i<_maxTransactionActive; i++) { if((_coordinatorState.hasCache(curr) || _coordinator.isReady()) && !_activeTx.containsKey(curr)) { TransactionAttempt attempt = new TransactionAttempt(curr, _rand.nextLong()); Object state = _coordinatorState.getState(curr, _initializer); _activeTx.put(curr, new TransactionStatus(attempt)); _collector.emit(TRANSACTION_BATCH_STREAM_ID, new Values(attempt, state, previousTransactionId(_currTransaction)), attempt); } curr = nextTransactionId(curr); } } } catch(FailedException e) { LOG.warn("Failed to get metadata for a transaction", e); } } @Override public Map getComponentConfiguration() { Config ret = new Config(); ret.setMaxTaskParallelism(1); return ret; } private static enum AttemptStatus { PROCESSING, PROCESSED, COMMITTING } private static class TransactionStatus { TransactionAttempt attempt; AttemptStatus status; public TransactionStatus(TransactionAttempt attempt) { this.attempt = attempt; this.status = AttemptStatus.PROCESSING; } @Override public String toString() { return attempt.toString() + " <" + status.toString() + ">"; } } private BigInteger nextTransactionId(BigInteger id) { return id.add(BigInteger.ONE); } private BigInteger previousTransactionId(BigInteger id) { if(id.equals(INIT_TXID)) { return null; } else { return id.subtract(BigInteger.ONE); } } private BigInteger getStoredCurrTransaction(TransactionalState state) { BigInteger ret = (BigInteger) state.getData(CURRENT_TX); if(ret==null) return INIT_TXID; else return ret; } private class StateInitializer implements RotatingTransactionalState.StateInitializer { @Override public Object init(BigInteger txid, Object lastState) { return _coordinator.initializeTransaction(txid, lastState); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/TransactionalTopologyBuilder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional; import backtype.storm.coordination.IBatchBolt; import backtype.storm.coordination.BatchBoltExecutor; import backtype.storm.Config; import backtype.storm.Constants; import backtype.storm.coordination.CoordinatedBolt; import backtype.storm.coordination.CoordinatedBolt.IdStreamSpec; import backtype.storm.coordination.CoordinatedBolt.SourceArgs; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.generated.StormTopology; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.topology.BaseConfigurationDeclarer; import backtype.storm.topology.BasicBoltExecutor; import backtype.storm.topology.BoltDeclarer; import backtype.storm.topology.IBasicBolt; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.InputDeclarer; import backtype.storm.topology.SpoutDeclarer; import backtype.storm.topology.TopologyBuilder; import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; import backtype.storm.transactional.partitioned.OpaquePartitionedTransactionalSpoutExecutor; import backtype.storm.transactional.partitioned.PartitionedTransactionalSpoutExecutor; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; /** * Trident subsumes the functionality provided by transactional topologies, so this * class is deprecated. * */ @Deprecated public class TransactionalTopologyBuilder { String _id; String _spoutId; ITransactionalSpout _spout; Map _bolts = new HashMap(); Integer _spoutParallelism; List _spoutConfs = new ArrayList(); // id is used to store the state of this transactionalspout in zookeeper // it would be very dangerous to have 2 topologies active with the same id in the same cluster public TransactionalTopologyBuilder(String id, String spoutId, ITransactionalSpout spout, Number spoutParallelism) { _id = id; _spoutId = spoutId; _spout = spout; _spoutParallelism = (spoutParallelism == null) ? null : spoutParallelism.intValue(); } public TransactionalTopologyBuilder(String id, String spoutId, ITransactionalSpout spout) { this(id, spoutId, spout, null); } public TransactionalTopologyBuilder(String id, String spoutId, IPartitionedTransactionalSpout spout, Number spoutParallelism) { this(id, spoutId, new PartitionedTransactionalSpoutExecutor(spout), spoutParallelism); } public TransactionalTopologyBuilder(String id, String spoutId, IPartitionedTransactionalSpout spout) { this(id, spoutId, spout, null); } public TransactionalTopologyBuilder(String id, String spoutId, IOpaquePartitionedTransactionalSpout spout, Number spoutParallelism) { this(id, spoutId, new OpaquePartitionedTransactionalSpoutExecutor(spout), spoutParallelism); } public TransactionalTopologyBuilder(String id, String spoutId, IOpaquePartitionedTransactionalSpout spout) { this(id, spoutId, spout, null); } public SpoutDeclarer getSpoutDeclarer() { return new SpoutDeclarerImpl(); } public BoltDeclarer setBolt(String id, IBatchBolt bolt) { return setBolt(id, bolt, null); } public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number parallelism) { return setBolt(id, new BatchBoltExecutor(bolt), parallelism, bolt instanceof ICommitter); } public BoltDeclarer setCommitterBolt(String id, IBatchBolt bolt) { return setCommitterBolt(id, bolt, null); } public BoltDeclarer setCommitterBolt(String id, IBatchBolt bolt, Number parallelism) { return setBolt(id, new BatchBoltExecutor(bolt), parallelism, true); } public BoltDeclarer setBolt(String id, IBasicBolt bolt) { return setBolt(id, bolt, null); } public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism) { return setBolt(id, new BasicBoltExecutor(bolt), parallelism, false); } private BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism, boolean committer) { Integer p = null; if(parallelism!=null) p = parallelism.intValue(); Component component = new Component(bolt, p, committer); _bolts.put(id, component); return new BoltDeclarerImpl(component); } public TopologyBuilder buildTopologyBuilder() { String coordinator = _spoutId + "/coordinator"; TopologyBuilder builder = new TopologyBuilder(); SpoutDeclarer declarer = builder.setSpout(coordinator, new TransactionalSpoutCoordinator(_spout)); for(Map conf: _spoutConfs) { declarer.addConfigurations(conf); } declarer.addConfiguration(Config.TOPOLOGY_TRANSACTIONAL_ID, _id); BoltDeclarer emitterDeclarer = builder.setBolt(_spoutId, new CoordinatedBolt(new TransactionalSpoutBatchExecutor(_spout), null, null), _spoutParallelism) .allGrouping(coordinator, TransactionalSpoutCoordinator.TRANSACTION_BATCH_STREAM_ID) .addConfiguration(Config.TOPOLOGY_TRANSACTIONAL_ID, _id); if(_spout instanceof ICommitterTransactionalSpout) { emitterDeclarer.allGrouping(coordinator, TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); } for(String id: _bolts.keySet()) { Component component = _bolts.get(id); Map coordinatedArgs = new HashMap(); for(String c: componentBoltSubscriptions(component)) { coordinatedArgs.put(c, SourceArgs.all()); } IdStreamSpec idSpec = null; if(component.committer) { idSpec = IdStreamSpec.makeDetectSpec(coordinator, TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); } BoltDeclarer input = builder.setBolt(id, new CoordinatedBolt(component.bolt, coordinatedArgs, idSpec), component.parallelism); for(Map conf: component.componentConfs) { input.addConfigurations(conf); } for(String c: componentBoltSubscriptions(component)) { input.directGrouping(c, Constants.COORDINATED_STREAM_ID); } for(InputDeclaration d: component.declarations) { d.declare(input); } if(component.committer) { input.allGrouping(coordinator, TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); } } return builder; } public StormTopology buildTopology() { return buildTopologyBuilder().createTopology(); } private Set componentBoltSubscriptions(Component component) { Set ret = new HashSet(); for(InputDeclaration d: component.declarations) { ret.add(d.getComponent()); } return ret; } private static class Component { public IRichBolt bolt; public Integer parallelism; public List declarations = new ArrayList(); public List componentConfs = new ArrayList(); public boolean committer; public Component(IRichBolt bolt, Integer parallelism, boolean committer) { this.bolt = bolt; this.parallelism = parallelism; this.committer = committer; } } private static interface InputDeclaration { void declare(InputDeclarer declarer); String getComponent(); } private class SpoutDeclarerImpl extends BaseConfigurationDeclarer implements SpoutDeclarer { @Override public SpoutDeclarer addConfigurations(Map conf) { _spoutConfs.add(conf); return this; } } private class BoltDeclarerImpl extends BaseConfigurationDeclarer implements BoltDeclarer { Component _component; public BoltDeclarerImpl(Component component) { _component = component; } @Override public BoltDeclarer fieldsGrouping(final String component, final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.fieldsGrouping(component, fields); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer fieldsGrouping(final String component, final String streamId, final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.fieldsGrouping(component, streamId, fields); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer globalGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.globalGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer globalGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.globalGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer shuffleGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.shuffleGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer shuffleGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.shuffleGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer localOrShuffleGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.localOrShuffleGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer localOrShuffleGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.localOrShuffleGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer noneGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.noneGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer noneGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.noneGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer allGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.allGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer allGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.allGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer directGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.directGrouping(component); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer directGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.directGrouping(component, streamId); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer customGrouping(final String component, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.customGrouping(component, grouping); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer customGrouping(final String component, final String streamId, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.customGrouping(component, streamId, grouping); } @Override public String getComponent() { return component; } }); return this; } @Override public BoltDeclarer grouping(final GlobalStreamId stream, final Grouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.grouping(stream, grouping); } @Override public String getComponent() { return stream.get_componentId(); } }); return this; } private void addDeclaration(InputDeclaration declaration) { _component.declarations.add(declaration); } @Override public BoltDeclarer addConfigurations(Map conf) { _component.componentConfs.add(conf); return this; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional.partitioned; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IComponent; import backtype.storm.transactional.TransactionAttempt; import java.util.Map; /** * This defines a transactional spout which does *not* necessarily * replay the same batch every time it emits a batch for a transaction id. */ public interface IOpaquePartitionedTransactionalSpout extends IComponent { public interface Coordinator { /** * Returns true if its ok to emit start a new transaction, false otherwise (will skip this transaction). * * You should sleep here if you want a delay between asking for the next transaction (this will be called * repeatedly in a loop). */ boolean isReady(); void close(); } public interface Emitter { /** * Emit a batch of tuples for a partition/transaction. * * Return the metadata describing this batch that will be used as lastPartitionMeta * for defining the parameters of the next batch. */ X emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, X lastPartitionMeta); int numPartitions(); void close(); } Emitter getEmitter(Map conf, TopologyContext context); Coordinator getCoordinator(Map conf, TopologyContext context); } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional.partitioned; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IComponent; import backtype.storm.transactional.TransactionAttempt; import backtype.storm.coordination.BatchOutputCollector; import java.util.Map; /** * This interface defines a transactional spout that reads its tuples from a partitioned set of * brokers. It automates the storing of metadata for each partition to ensure that the same batch * is always emitted for the same transaction id. The partition metadata is stored in Zookeeper. */ public interface IPartitionedTransactionalSpout extends IComponent { public interface Coordinator { /** * Return the number of partitions currently in the source of data. The idea is * is that if a new partition is added and a prior transaction is replayed, it doesn't * emit tuples for the new partition because it knows how many partitions were in * that transaction. */ int numPartitions(); /** * Returns true if its ok to emit start a new transaction, false otherwise (will skip this transaction). * * You should sleep here if you want a delay between asking for the next transaction (this will be called * repeatedly in a loop). */ boolean isReady(); void close(); } public interface Emitter { /** * Emit a batch of tuples for a partition/transaction that's never been emitted before. * Return the metadata that can be used to reconstruct this partition/batch in the future. */ X emitPartitionBatchNew(TransactionAttempt tx, BatchOutputCollector collector, int partition, X lastPartitionMeta); /** * Emit a batch of tuples for a partition/transaction that has been emitted before, using * the metadata created when it was first emitted. */ void emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, X partitionMeta); void close(); } Coordinator getCoordinator(Map conf, TopologyContext context); Emitter getEmitter(Map conf, TopologyContext context); } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional.partitioned; import backtype.storm.Config; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.transactional.ICommitterTransactionalSpout; import backtype.storm.transactional.ITransactionalSpout; import backtype.storm.transactional.TransactionAttempt; import backtype.storm.transactional.state.RotatingTransactionalState; import backtype.storm.transactional.state.TransactionalState; import java.math.BigInteger; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; public class OpaquePartitionedTransactionalSpoutExecutor implements ICommitterTransactionalSpout { IOpaquePartitionedTransactionalSpout _spout; public class Coordinator implements ITransactionalSpout.Coordinator { IOpaquePartitionedTransactionalSpout.Coordinator _coordinator; public Coordinator(Map conf, TopologyContext context) { _coordinator = _spout.getCoordinator(conf, context); } @Override public Object initializeTransaction(BigInteger txid, Object prevMetadata) { return null; } @Override public boolean isReady() { return _coordinator.isReady(); } @Override public void close() { _coordinator.close(); } } public class Emitter implements ICommitterTransactionalSpout.Emitter { IOpaquePartitionedTransactionalSpout.Emitter _emitter; TransactionalState _state; TreeMap> _cachedMetas = new TreeMap>(); Map _partitionStates = new HashMap(); int _index; int _numTasks; public Emitter(Map conf, TopologyContext context) { _emitter = _spout.getEmitter(conf, context); _index = context.getThisTaskIndex(); _numTasks = context.getComponentTasks(context.getThisComponentId()).size(); _state = TransactionalState.newUserState(conf, (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), getComponentConfiguration()); List existingPartitions = _state.list(""); for(String p: existingPartitions) { int partition = Integer.parseInt(p); if((partition - _index) % _numTasks == 0) { _partitionStates.put(partition, new RotatingTransactionalState(_state, p)); } } } @Override public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, BatchOutputCollector collector) { Map metas = new HashMap(); _cachedMetas.put(tx.getTransactionId(), metas); int partitions = _emitter.numPartitions(); Entry> entry = _cachedMetas.lowerEntry(tx.getTransactionId()); Map prevCached; if(entry!=null) { prevCached = entry.getValue(); } else { prevCached = new HashMap(); } for(int i=_index; i < partitions; i+=_numTasks) { RotatingTransactionalState state = _partitionStates.get(i); if(state==null) { state = new RotatingTransactionalState(_state, "" + i); _partitionStates.put(i, state); } state.removeState(tx.getTransactionId()); Object lastMeta = prevCached.get(i); if(lastMeta==null) lastMeta = state.getLastState(); Object meta = _emitter.emitPartitionBatch(tx, collector, i, lastMeta); metas.put(i, meta); } } @Override public void cleanupBefore(BigInteger txid) { for(RotatingTransactionalState state: _partitionStates.values()) { state.cleanupBefore(txid); } } @Override public void commit(TransactionAttempt attempt) { BigInteger txid = attempt.getTransactionId(); Map metas = _cachedMetas.remove(txid); for(Integer partition: metas.keySet()) { Object meta = metas.get(partition); _partitionStates.get(partition).overrideState(txid, meta); } } @Override public void close() { _emitter.close(); } } public OpaquePartitionedTransactionalSpoutExecutor(IOpaquePartitionedTransactionalSpout spout) { _spout = spout; } @Override public ITransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { return new Coordinator(conf, context); } @Override public ICommitterTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { return new Emitter(conf, context); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { _spout.declareOutputFields(declarer); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional.partitioned; import backtype.storm.Config; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.transactional.ITransactionalSpout; import backtype.storm.transactional.TransactionAttempt; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.transactional.state.RotatingTransactionalState; import backtype.storm.transactional.state.TransactionalState; import java.math.BigInteger; import java.util.HashMap; import java.util.Map; public class PartitionedTransactionalSpoutExecutor implements ITransactionalSpout { IPartitionedTransactionalSpout _spout; public PartitionedTransactionalSpoutExecutor(IPartitionedTransactionalSpout spout) { _spout = spout; } public IPartitionedTransactionalSpout getPartitionedSpout() { return _spout; } class Coordinator implements ITransactionalSpout.Coordinator { private IPartitionedTransactionalSpout.Coordinator _coordinator; public Coordinator(Map conf, TopologyContext context) { _coordinator = _spout.getCoordinator(conf, context); } @Override public Integer initializeTransaction(BigInteger txid, Integer prevMetadata) { return _coordinator.numPartitions(); } @Override public boolean isReady() { return _coordinator.isReady(); } @Override public void close() { _coordinator.close(); } } class Emitter implements ITransactionalSpout.Emitter { private IPartitionedTransactionalSpout.Emitter _emitter; private TransactionalState _state; private Map _partitionStates = new HashMap(); private int _index; private int _numTasks; public Emitter(Map conf, TopologyContext context) { _emitter = _spout.getEmitter(conf, context); _state = TransactionalState.newUserState(conf, (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), getComponentConfiguration()); _index = context.getThisTaskIndex(); _numTasks = context.getComponentTasks(context.getThisComponentId()).size(); } @Override public void emitBatch(final TransactionAttempt tx, final Integer partitions, final BatchOutputCollector collector) { for(int i=_index; i < partitions; i+=_numTasks) { if(!_partitionStates.containsKey(i)) { _partitionStates.put(i, new RotatingTransactionalState(_state, "" + i)); } RotatingTransactionalState state = _partitionStates.get(i); final int partition = i; Object meta = state.getStateOrCreate(tx.getTransactionId(), new RotatingTransactionalState.StateInitializer() { @Override public Object init(BigInteger txid, Object lastState) { return _emitter.emitPartitionBatchNew(tx, collector, partition, lastState); } }); // it's null if one of: // a) a later transaction batch was emitted before this, so we should skip this batch // b) if didn't exist and was created (in which case the StateInitializer was invoked and // it was emitted if(meta!=null) { _emitter.emitPartitionBatch(tx, collector, partition, meta); } } } @Override public void cleanupBefore(BigInteger txid) { for(RotatingTransactionalState state: _partitionStates.values()) { state.cleanupBefore(txid); } } @Override public void close() { _state.close(); _emitter.close(); } } @Override public ITransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { return new Coordinator(conf, context); } @Override public ITransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { return new Emitter(conf, context); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { _spout.declareOutputFields(declarer); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/state/RotatingTransactionalState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional.state; import backtype.storm.transactional.TransactionalSpoutCoordinator; import java.math.BigInteger; import java.util.HashSet; import java.util.List; import java.util.SortedMap; import java.util.TreeMap; /** * A map from txid to a value. Automatically deletes txids that have been committed. */ public class RotatingTransactionalState { public static interface StateInitializer { Object init(BigInteger txid, Object lastState); } private TransactionalState _state; private String _subdir; private boolean _strictOrder; private TreeMap _curr = new TreeMap(); public RotatingTransactionalState(TransactionalState state, String subdir, boolean strictOrder) { _state = state; _subdir = subdir; _strictOrder = strictOrder; state.mkdir(subdir); sync(); } public RotatingTransactionalState(TransactionalState state, String subdir) { this(state, subdir, false); } public Object getLastState() { if(_curr.isEmpty()) return null; else return _curr.lastEntry().getValue(); } public void overrideState(BigInteger txid, Object state) { _state.setData(txPath(txid), state); _curr.put(txid, state); } public void removeState(BigInteger txid) { if(_curr.containsKey(txid)) { _curr.remove(txid); _state.delete(txPath(txid)); } } public Object getState(BigInteger txid, StateInitializer init) { if(!_curr.containsKey(txid)) { SortedMap prevMap = _curr.headMap(txid); SortedMap afterMap = _curr.tailMap(txid); BigInteger prev = null; if(!prevMap.isEmpty()) prev = prevMap.lastKey(); if(_strictOrder) { if(prev==null && !txid.equals(TransactionalSpoutCoordinator.INIT_TXID)) { throw new IllegalStateException("Trying to initialize transaction for which there should be a previous state"); } if(prev!=null && !prev.equals(txid.subtract(BigInteger.ONE))) { throw new IllegalStateException("Expecting previous txid state to be the previous transaction"); } if(!afterMap.isEmpty()) { throw new IllegalStateException("Expecting tx state to be initialized in strict order but there are txids after that have state"); } } Object data; if(afterMap.isEmpty()) { Object prevData; if(prev!=null) { prevData = _curr.get(prev); } else { prevData = null; } data = init.init(txid, prevData); } else { data = null; } _curr.put(txid, data); _state.setData(txPath(txid), data); } return _curr.get(txid); } public boolean hasCache(BigInteger txid) { return _curr.containsKey(txid); } /** * Returns null if it was created, the value otherwise. */ public Object getStateOrCreate(BigInteger txid, StateInitializer init) { if(_curr.containsKey(txid)) { return _curr.get(txid); } else { getState(txid, init); return null; } } public void cleanupBefore(BigInteger txid) { SortedMap toDelete = _curr.headMap(txid); for(BigInteger tx: new HashSet(toDelete.keySet())) { _curr.remove(tx); _state.delete(txPath(tx)); } } private void sync() { List txids = _state.list(_subdir); for(String txid_s: txids) { Object data = _state.getData(txPath(txid_s)); _curr.put(new BigInteger(txid_s), data); } } private String txPath(BigInteger tx) { return txPath(tx.toString()); } private String txPath(String tx) { return _subdir + "/" + tx; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.transactional.state; import backtype.storm.Config; import backtype.storm.serialization.KryoValuesDeserializer; import backtype.storm.serialization.KryoValuesSerializer; import backtype.storm.utils.Utils; import com.netflix.curator.framework.CuratorFramework; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; public class TransactionalState { CuratorFramework _curator; KryoValuesSerializer _ser; KryoValuesDeserializer _des; public static TransactionalState newUserState(Map conf, String id, Map componentConf) { return new TransactionalState(conf, id, componentConf, "user"); } public static TransactionalState newCoordinatorState(Map conf, String id, Map componentConf) { return new TransactionalState(conf, id, componentConf, "coordinator"); } protected TransactionalState(Map conf, String id, Map componentConf, String subroot) { try { conf = new HashMap(conf); // ensure that the serialization registrations are consistent with the declarations in this spout if(componentConf!=null) { conf.put(Config.TOPOLOGY_KRYO_REGISTER, componentConf .get(Config.TOPOLOGY_KRYO_REGISTER)); } String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) + "/" + id + "/" + subroot; List servers = (List) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS); Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT); CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port); try { initter.create().creatingParentsIfNeeded().forPath(rootDir); } catch(KeeperException.NodeExistsException e) { } initter.close(); _curator = Utils.newCuratorStarted(conf, servers, port, rootDir); _ser = new KryoValuesSerializer(conf); _des = new KryoValuesDeserializer(conf); } catch (Exception e) { throw new RuntimeException(e); } } public void setData(String path, Object obj) { path = "/" + path; byte[] ser = _ser.serializeObject(obj); try { if(_curator.checkExists().forPath(path)!=null) { _curator.setData().forPath(path, ser); } else { _curator.create() .creatingParentsIfNeeded() .withMode(CreateMode.PERSISTENT) .forPath(path, ser); } } catch(Exception e) { throw new RuntimeException(e); } } public void delete(String path) { path = "/" + path; try { _curator.delete().forPath(path); } catch (Exception e) { throw new RuntimeException(e); } } public List list(String path) { path = "/" + path; try { if(_curator.checkExists().forPath(path)==null) { return new ArrayList(); } else { return _curator.getChildren().forPath(path); } } catch(Exception e) { throw new RuntimeException(e); } } public void mkdir(String path) { setData(path, 7); } public Object getData(String path) { path = "/" + path; try { if(_curator.checkExists().forPath(path)!=null) { return _des.deserializeObject(_curator.getData().forPath(path)); } else { return null; } } catch(Exception e) { throw new RuntimeException(e); } } public void close() { _curator.close(); } private Object getWithBackup(Map amap, Object primary, Object backup) { Object ret = amap.get(primary); if(ret==null) return amap.get(backup); return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/tuple/Fields.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.tuple; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.io.Serializable; public class Fields implements Iterable, Serializable { private List _fields; private Map _index = new HashMap(); public Fields(String... fields) { this(Arrays.asList(fields)); } public Fields(List fields) { _fields = new ArrayList(fields.size()); for (String field : fields) { if (_fields.contains(field)) throw new IllegalArgumentException( String.format("duplicate field '%s'", field) ); _fields.add(field); } index(); } public List select(Fields selector, List tuple) { List ret = new ArrayList(selector.size()); for(String s: selector) { ret.add(tuple.get(_index.get(s))); } return ret; } public List toList() { return new ArrayList(_fields); } public int size() { return _fields.size(); } public String get(int index) { return _fields.get(index); } public Iterator iterator() { return _fields.iterator(); } /** * Returns the position of the specified field. */ public int fieldIndex(String field) { Integer ret = _index.get(field); if(ret==null) { throw new IllegalArgumentException(field + " does not exist"); } return ret; } /** * Returns true if this contains the specified name of the field. */ public boolean contains(String field) { return _index.containsKey(field); } private void index() { for(int i=0; i<_fields.size(); i++) { _index.put(_fields.get(i), i); } } @Override public String toString() { return _fields.toString(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/tuple/MessageId.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.tuple; import backtype.storm.utils.Utils; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; import java.util.Random; import java.util.Set; public class MessageId { private Map _anchorsToIds; public static long generateId(Random rand) { return rand.nextLong(); } public static MessageId makeUnanchored() { return makeId(new HashMap()); } public static MessageId makeId(Map anchorsToIds) { return new MessageId(anchorsToIds); } public static MessageId makeRootId(long id, long val) { Map anchorsToIds = new HashMap(); anchorsToIds.put(id, val); return new MessageId(anchorsToIds); } protected MessageId(Map anchorsToIds) { _anchorsToIds = anchorsToIds; } public Map getAnchorsToIds() { return _anchorsToIds; } public Set getAnchors() { return _anchorsToIds.keySet(); } @Override public int hashCode() { return _anchorsToIds.hashCode(); } @Override public boolean equals(Object other) { if(other instanceof MessageId) { return _anchorsToIds.equals(((MessageId) other)._anchorsToIds); } else { return false; } } @Override public String toString() { return _anchorsToIds.toString(); } public void serialize(Output out) throws IOException { out.writeInt(_anchorsToIds.size(), true); for(Entry anchorToId: _anchorsToIds.entrySet()) { out.writeLong(anchorToId.getKey()); out.writeLong(anchorToId.getValue()); } } public static MessageId deserialize(Input in) throws IOException { int numAnchors = in.readInt(true); Map anchorsToIds = new HashMap(); for(int i=0; i getValues(); /** * Gets the names of the fields in this tuple. */ public Fields getFields(); /** * Returns a subset of the tuple based on the fields selector. */ public List select(Fields selector); /** * Returns the global stream id (component + stream) of this tuple. */ public GlobalStreamId getSourceGlobalStreamid(); /** * Gets the id of the component that created this tuple. */ public String getSourceComponent(); /** * Gets the id of the task that created this tuple. */ public int getSourceTask(); /** * Gets the id of the stream that this tuple was emitted to. */ public String getSourceStreamId(); /** * Gets the message id that associated with this tuple. */ public MessageId getMessageId(); } ================================================ FILE: storm-core/src/jvm/backtype/storm/tuple/TupleImpl.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.tuple; import backtype.storm.generated.GlobalStreamId; import backtype.storm.task.GeneralTopologyContext; import backtype.storm.utils.IndifferentAccessMap; import clojure.lang.ASeq; import clojure.lang.Counted; import clojure.lang.IMeta; import clojure.lang.IPersistentMap; import clojure.lang.ISeq; import clojure.lang.Indexed; import clojure.lang.Keyword; import clojure.lang.MapEntry; import clojure.lang.Obj; import clojure.lang.PersistentArrayMap; import clojure.lang.Seqable; import clojure.lang.Symbol; import java.util.List; public class TupleImpl extends IndifferentAccessMap implements Seqable, Indexed, IMeta, Tuple { private List values; private int taskId; private String streamId; private GeneralTopologyContext context; private MessageId id; private IPersistentMap _meta = null; public TupleImpl(GeneralTopologyContext context, List values, int taskId, String streamId, MessageId id) { this.values = values; this.taskId = taskId; this.streamId = streamId; this.id = id; this.context = context; String componentId = context.getComponentId(taskId); Fields schema = context.getComponentOutputFields(componentId, streamId); if(values.size()!=schema.size()) { throw new IllegalArgumentException( "Tuple created with wrong number of fields. " + "Expected " + schema.size() + " fields but got " + values.size() + " fields"); } } public TupleImpl(GeneralTopologyContext context, List values, int taskId, String streamId) { this(context, values, taskId, streamId, MessageId.makeUnanchored()); } Long _processSampleStartTime = null; Long _executeSampleStartTime = null; public void setProcessSampleStartTime(long ms) { _processSampleStartTime = ms; } public Long getProcessSampleStartTime() { return _processSampleStartTime; } public void setExecuteSampleStartTime(long ms) { _executeSampleStartTime = ms; } public Long getExecuteSampleStartTime() { return _executeSampleStartTime; } long _outAckVal = 0; public void updateAckVal(long val) { _outAckVal = _outAckVal ^ val; } public long getAckVal() { return _outAckVal; } public int size() { return values.size(); } public int fieldIndex(String field) { return getFields().fieldIndex(field); } public boolean contains(String field) { return getFields().contains(field); } public Object getValue(int i) { return values.get(i); } public String getString(int i) { return (String) values.get(i); } public Integer getInteger(int i) { return (Integer) values.get(i); } public Long getLong(int i) { return (Long) values.get(i); } public Boolean getBoolean(int i) { return (Boolean) values.get(i); } public Short getShort(int i) { return (Short) values.get(i); } public Byte getByte(int i) { return (Byte) values.get(i); } public Double getDouble(int i) { return (Double) values.get(i); } public Float getFloat(int i) { return (Float) values.get(i); } public byte[] getBinary(int i) { return (byte[]) values.get(i); } public Object getValueByField(String field) { return values.get(fieldIndex(field)); } public String getStringByField(String field) { return (String) values.get(fieldIndex(field)); } public Integer getIntegerByField(String field) { return (Integer) values.get(fieldIndex(field)); } public Long getLongByField(String field) { return (Long) values.get(fieldIndex(field)); } public Boolean getBooleanByField(String field) { return (Boolean) values.get(fieldIndex(field)); } public Short getShortByField(String field) { return (Short) values.get(fieldIndex(field)); } public Byte getByteByField(String field) { return (Byte) values.get(fieldIndex(field)); } public Double getDoubleByField(String field) { return (Double) values.get(fieldIndex(field)); } public Float getFloatByField(String field) { return (Float) values.get(fieldIndex(field)); } public byte[] getBinaryByField(String field) { return (byte[]) values.get(fieldIndex(field)); } public List getValues() { return values; } public Fields getFields() { return context.getComponentOutputFields(getSourceComponent(), getSourceStreamId()); } public List select(Fields selector) { return getFields().select(selector, values); } public GlobalStreamId getSourceGlobalStreamid() { return new GlobalStreamId(getSourceComponent(), streamId); } public String getSourceComponent() { return context.getComponentId(taskId); } public int getSourceTask() { return taskId; } public String getSourceStreamId() { return streamId; } public MessageId getMessageId() { return id; } @Override public String toString() { return "source: " + getSourceComponent() + ":" + taskId + ", stream: " + streamId + ", id: "+ id.toString() + ", " + values.toString(); } @Override public boolean equals(Object other) { return this == other; } @Override public int hashCode() { return System.identityHashCode(this); } private final Keyword makeKeyword(String name) { return Keyword.intern(Symbol.create(name)); } /* ILookup */ @Override public Object valAt(Object o) { try { if(o instanceof Keyword) { return getValueByField(((Keyword) o).getName()); } else if(o instanceof String) { return getValueByField((String) o); } } catch(IllegalArgumentException e) { } return null; } /* Seqable */ public ISeq seq() { if(values.size() > 0) { return new Seq(getFields().toList(), values, 0); } return null; } static class Seq extends ASeq implements Counted { final List fields; final List values; final int i; Seq(List fields, List values, int i) { this.fields = fields; this.values = values; assert i >= 0; this.i = i; } public Seq(IPersistentMap meta, List fields, List values, int i) { super(meta); this.fields= fields; this.values = values; assert i >= 0; this.i = i; } public Object first() { return new MapEntry(fields.get(i), values.get(i)); } public ISeq next() { if(i+1 < fields.size()) { return new Seq(fields, values, i+1); } return null; } public int count() { assert fields.size() -i >= 0 : "index out of bounds"; // i being the position in the fields of this seq, the remainder of the seq is the size return fields.size() -i; } public Obj withMeta(IPersistentMap meta) { return new Seq(meta, fields, values, i); } } /* Indexed */ public Object nth(int i) { if(i < values.size()) { return values.get(i); } else { return null; } } public Object nth(int i, Object notfound) { Object ret = nth(i); if(ret==null) ret = notfound; return ret; } /* Counted */ public int count() { return values.size(); } /* IMeta */ public IPersistentMap meta() { if(_meta==null) { _meta = new PersistentArrayMap( new Object[] { makeKeyword("stream"), getSourceStreamId(), makeKeyword("component"), getSourceComponent(), makeKeyword("task"), getSourceTask()}); } return _meta; } private PersistentArrayMap toMap() { Object array[] = new Object[values.size()*2]; List fields = getFields().toList(); for(int i=0; i < values.size(); i++) { array[i*2] = fields.get(i); array[(i*2)+1] = values.get(i); } return new PersistentArrayMap(array); } public IPersistentMap getMap() { if(_map==null) { setMap(toMap()); } return _map; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/tuple/Values.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.tuple; import java.util.ArrayList; /** * A convenience class for making tuple values using new Values("field1", 2, 3) * syntax. */ public class Values extends ArrayList{ public Values() { } public Values(Object... vals) { super(vals.length); for(Object o: vals) { add(o); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/BufferFileInputStream.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.util.Arrays; public class BufferFileInputStream { byte[] buffer; FileInputStream stream; public BufferFileInputStream(String file, int bufferSize) throws FileNotFoundException { stream = new FileInputStream(file); buffer = new byte[bufferSize]; } public BufferFileInputStream(String file) throws FileNotFoundException { this(file, 15*1024); } public byte[] read() throws IOException { int length = stream.read(buffer); if(length==-1) { close(); return new byte[0]; } else if(length==buffer.length) { return buffer; } else { return Arrays.copyOf(buffer, length); } } public void close() throws IOException { stream.close(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/CRC32OutputStream.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.io.IOException; import java.io.OutputStream; import java.util.zip.CRC32; public class CRC32OutputStream extends OutputStream { private CRC32 hasher; public CRC32OutputStream() { hasher = new CRC32(); } public long getValue() { return hasher.getValue(); } @Override public void write(int i) throws IOException { hasher.update(i); } @Override public void write(byte[] bytes, int start, int end) throws IOException { hasher.update(bytes, start, end); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/ClojureTimerTask.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import clojure.lang.IFn; import java.util.TimerTask; public class ClojureTimerTask extends TimerTask { IFn _afn; public ClojureTimerTask(IFn afn) { super(); _afn = afn; } @Override public void run() { _afn.run(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/Container.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.io.Serializable; public class Container implements Serializable { public Object object; } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/DRPCClient.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import backtype.storm.generated.DRPCExecutionException; import backtype.storm.generated.DistributedRPC; import org.apache.thrift7.TException; import org.apache.thrift7.protocol.TBinaryProtocol; import org.apache.thrift7.transport.TFramedTransport; import org.apache.thrift7.transport.TSocket; import org.apache.thrift7.transport.TTransport; public class DRPCClient implements DistributedRPC.Iface { private TTransport conn; private DistributedRPC.Client client; private String host; private int port; private Integer timeout; public DRPCClient(String host, int port, Integer timeout) { try { this.host = host; this.port = port; this.timeout = timeout; connect(); } catch(TException e) { throw new RuntimeException(e); } } public DRPCClient(String host, int port) { this(host, port, null); } private void connect() throws TException { TSocket socket = new TSocket(host, port); if(timeout!=null) { socket.setTimeout(timeout); } conn = new TFramedTransport(socket); client = new DistributedRPC.Client(new TBinaryProtocol(conn)); conn.open(); } public String getHost() { return host; } public int getPort() { return port; } public String execute(String func, String args) throws TException, DRPCExecutionException { try { if(client==null) connect(); return client.execute(func, args); } catch(TException e) { client = null; throw e; } catch(DRPCExecutionException e) { client = null; throw e; } } public void close() { conn.close(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import com.lmax.disruptor.AlertException; import com.lmax.disruptor.ClaimStrategy; import com.lmax.disruptor.EventFactory; import com.lmax.disruptor.EventHandler; import com.lmax.disruptor.InsufficientCapacityException; import com.lmax.disruptor.RingBuffer; import com.lmax.disruptor.Sequence; import com.lmax.disruptor.SequenceBarrier; import com.lmax.disruptor.SingleThreadedClaimStrategy; import com.lmax.disruptor.WaitStrategy; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; import java.util.HashMap; import java.util.Map; import backtype.storm.metric.api.IStatefulObject; import java.util.logging.Level; import java.util.logging.Logger; /** * * A single consumer queue that uses the LMAX Disruptor. They key to the performance is * the ability to catch up to the producer by processing tuples in batches. */ public class DisruptorQueue implements IStatefulObject { static final Object FLUSH_CACHE = new Object(); static final Object INTERRUPT = new Object(); RingBuffer _buffer; Sequence _consumer; SequenceBarrier _barrier; // TODO: consider having a threadlocal cache of this variable to speed up reads? volatile boolean consumerStartedFlag = false; ConcurrentLinkedQueue _cache = new ConcurrentLinkedQueue(); public DisruptorQueue(ClaimStrategy claim, WaitStrategy wait) { _buffer = new RingBuffer(new ObjectEventFactory(), claim, wait); _consumer = new Sequence(); _barrier = _buffer.newBarrier(); _buffer.setGatingSequences(_consumer); if(claim instanceof SingleThreadedClaimStrategy) { consumerStartedFlag = true; } } public void consumeBatch(EventHandler handler) { consumeBatchToCursor(_barrier.getCursor(), handler); } public void haltWithInterrupt() { publish(INTERRUPT); } public void consumeBatchWhenAvailable(EventHandler handler) { try { final long nextSequence = _consumer.get() + 1; final long availableSequence = _barrier.waitFor(nextSequence, 10, TimeUnit.MILLISECONDS); if(availableSequence >= nextSequence) { consumeBatchToCursor(availableSequence, handler); } } catch (AlertException e) { throw new RuntimeException(e); } catch (InterruptedException e) { throw new RuntimeException(e); } } private void consumeBatchToCursor(long cursor, EventHandler handler) { for(long curr = _consumer.get() + 1; curr <= cursor; curr++) { try { MutableObject mo = _buffer.get(curr); Object o = mo.o; mo.setObject(null); if(o==FLUSH_CACHE) { Object c = null; while(true) { c = _cache.poll(); if(c==null) break; else handler.onEvent(c, curr, true); } } else if(o==INTERRUPT) { throw new InterruptedException("Disruptor processing interrupted"); } else { handler.onEvent(o, curr, curr == cursor); } } catch (Exception e) { throw new RuntimeException(e); } } //TODO: only set this if the consumer cursor has changed? _consumer.set(cursor); } /* * Caches until consumerStarted is called, upon which the cache is flushed to the consumer */ public void publish(Object obj) { try { publish(obj, true); } catch (InsufficientCapacityException ex) { throw new RuntimeException("This code should be unreachable!"); } } public void tryPublish(Object obj) throws InsufficientCapacityException { publish(obj, false); } public void publish(Object obj, boolean block) throws InsufficientCapacityException { if(consumerStartedFlag) { final long id; if(block) { id = _buffer.next(); } else { id = _buffer.tryNext(1); } final MutableObject m = _buffer.get(id); m.setObject(obj); _buffer.publish(id); } else { _cache.add(obj); if(consumerStartedFlag) flushCache(); } } public void consumerStarted() { if(!consumerStartedFlag) { consumerStartedFlag = true; flushCache(); } } private void flushCache() { publish(FLUSH_CACHE); } public long population() { return (writePos() - readPos()); } public long capacity() { return _buffer.getBufferSize(); } public long writePos() { return _buffer.getCursor(); } public long readPos() { return _consumer.get(); } public float pctFull() { return (1.0F * population() / capacity()); } @Override public Object getState() { Map state = new HashMap(); // get readPos then writePos so it's never an under-estimate long rp = readPos(); long wp = writePos(); state.put("capacity", capacity()); state.put("population", wp - rp); state.put("write_pos", wp); state.put("read_pos", rp); return state; } public static class ObjectEventFactory implements EventFactory { @Override public MutableObject newInstance() { return new MutableObject(); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/IndifferentAccessMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import clojure.lang.ILookup; import clojure.lang.ISeq; import clojure.lang.AFn; import clojure.lang.IPersistentMap; import clojure.lang.PersistentArrayMap; import clojure.lang.IMapEntry; import clojure.lang.IPersistentCollection; import clojure.lang.Keyword; import java.util.Iterator; import java.util.Map; import java.util.Collection; import java.util.Set; public class IndifferentAccessMap extends AFn implements ILookup, IPersistentMap, Map { protected IPersistentMap _map; protected IndifferentAccessMap() { } public IndifferentAccessMap(IPersistentMap map) { setMap(map); } public IPersistentMap getMap() { return _map; } public IPersistentMap setMap(IPersistentMap map) { _map = map; return _map; } public int size() { return ((Map) getMap()).size(); } public int count() { return size(); } public ISeq seq() { return getMap().seq(); } @Override public Object valAt(Object o) { if(o instanceof Keyword) { return valAt(((Keyword) o).getName()); } return getMap().valAt(o); } @Override public Object valAt(Object o, Object def) { Object ret = valAt(o); if(ret==null) ret = def; return ret; } /* IFn */ @Override public Object invoke(Object o) { return valAt(o); } @Override public Object invoke(Object o, Object notfound) { return valAt(o, notfound); } /* IPersistentMap */ /* Naive implementation, but it might be good enough */ public IPersistentMap assoc(Object k, Object v) { if(k instanceof Keyword) return assoc(((Keyword) k).getName(), v); return new IndifferentAccessMap(getMap().assoc(k, v)); } public IPersistentMap assocEx(Object k, Object v) { if(k instanceof Keyword) return assocEx(((Keyword) k).getName(), v); return new IndifferentAccessMap(getMap().assocEx(k, v)); } public IPersistentMap without(Object k) { if(k instanceof Keyword) return without(((Keyword) k).getName()); return new IndifferentAccessMap(getMap().without(k)); } public boolean containsKey(Object k) { if(k instanceof Keyword) return containsKey(((Keyword) k).getName()); return getMap().containsKey(k); } public IMapEntry entryAt(Object k) { if(k instanceof Keyword) return entryAt(((Keyword) k).getName()); return getMap().entryAt(k); } public IPersistentCollection cons(Object o) { return getMap().cons(o); } public IPersistentCollection empty() { return new IndifferentAccessMap(PersistentArrayMap.EMPTY); } public boolean equiv(Object o) { return getMap().equiv(o); } public Iterator iterator() { return getMap().iterator(); } /* Map */ public boolean containsValue(Object v) { return ((Map) getMap()).containsValue(v); } public Set entrySet() { return ((Map) getMap()).entrySet(); } public Object get(Object k) { return valAt(k); } public boolean isEmpty() { return ((Map) getMap()).isEmpty(); } public Set keySet() { return ((Map) getMap()).keySet(); } public Collection values() { return ((Map) getMap()).values(); } /* Not implemented */ public void clear() { throw new UnsupportedOperationException(); } public Object put(Object k, Object v) { throw new UnsupportedOperationException(); } public void putAll(Map m) { throw new UnsupportedOperationException(); } public Object remove(Object k) { throw new UnsupportedOperationException(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/InprocMessaging.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.HashMap; import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; public class InprocMessaging { private static Map> _queues = new HashMap>(); private static final Object _lock = new Object(); private static int port = 1; public static int acquireNewPort() { int ret; synchronized(_lock) { ret = port; port++; } return ret; } public static void sendMessage(int port, Object msg) { getQueue(port).add(msg); } public static Object takeMessage(int port) throws InterruptedException { return getQueue(port).take(); } public static Object pollMessage(int port) { return getQueue(port).poll(); } private static LinkedBlockingQueue getQueue(int port) { synchronized(_lock) { if(!_queues.containsKey(port)) { _queues.put(port, new LinkedBlockingQueue()); } return _queues.get(port); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.Semaphore; public class KeyedRoundRobinQueue { private final Object _lock = new Object(); private Semaphore _size = new Semaphore(0); private Map> _queues = new HashMap>(); private List _keyOrder = new ArrayList(); private int _currIndex = 0; public void add(Object key, V val) { synchronized(_lock) { Queue queue = _queues.get(key); if(queue==null) { queue = new LinkedList(); _queues.put(key, queue); _keyOrder.add(key); } queue.add(val); } _size.release(); } public V take() throws InterruptedException { _size.acquire(); synchronized(_lock) { Object key = _keyOrder.get(_currIndex); Queue queue = _queues.get(key); V ret = queue.remove(); if(queue.isEmpty()) { _keyOrder.remove(_currIndex); _queues.remove(key); if(_keyOrder.size()==0) { _currIndex = 0; } else { _currIndex = _currIndex % _keyOrder.size(); } } else { _currIndex = (_currIndex + 1) % _keyOrder.size(); } return ret; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/ListDelegate.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.Collection; import java.util.Iterator; import java.util.ArrayList; import java.util.List; import java.util.ListIterator; public class ListDelegate implements List { private List _delegate; public ListDelegate() { _delegate = new ArrayList(); } public void setDelegate(List delegate) { _delegate = delegate; } public List getDelegate() { return _delegate; } @Override public int size() { return _delegate.size(); } @Override public boolean isEmpty() { return _delegate.isEmpty(); } @Override public boolean contains(Object o) { return _delegate.contains(o); } @Override public Iterator iterator() { return _delegate.iterator(); } @Override public Object[] toArray() { return _delegate.toArray(); } @Override public T[] toArray(T[] ts) { return _delegate.toArray(ts); } @Override public boolean add(Object e) { return _delegate.add(e); } @Override public boolean remove(Object o) { return _delegate.remove(o); } @Override public boolean containsAll(Collection clctn) { return _delegate.containsAll(clctn); } @Override public boolean addAll(Collection clctn) { return _delegate.addAll(clctn); } @Override public boolean addAll(int i, Collection clctn) { return _delegate.addAll(i, clctn); } @Override public boolean removeAll(Collection clctn) { return _delegate.removeAll(clctn); } @Override public boolean retainAll(Collection clctn) { return _delegate.retainAll(clctn); } @Override public void clear() { _delegate.clear(); } @Override public Object get(int i) { return _delegate.get(i); } @Override public Object set(int i, Object e) { return _delegate.set(i, e); } @Override public void add(int i, Object e) { _delegate.add(i, e); } @Override public Object remove(int i) { return _delegate.remove(i); } @Override public int indexOf(Object o) { return _delegate.indexOf(o); } @Override public int lastIndexOf(Object o) { return _delegate.lastIndexOf(o); } @Override public ListIterator listIterator() { return _delegate.listIterator(); } @Override public ListIterator listIterator(int i) { return _delegate.listIterator(i); } @Override public List subList(int i, int i1) { return _delegate.subList(i, i1); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/LocalState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import org.apache.commons.io.FileUtils; import java.io.File; import java.util.Map; import java.util.HashMap; import java.io.IOException; /** * A simple, durable, atomic K/V database. *Very inefficient*, should only be used for occasional reads/writes. * Every read/write hits disk. */ public class LocalState { private VersionedStore _vs; public LocalState(String backingDir) throws IOException { _vs = new VersionedStore(backingDir); } public synchronized Map snapshot() throws IOException { int attempts = 0; while(true) { String latestPath = _vs.mostRecentVersionPath(); if(latestPath==null) return new HashMap(); try { return (Map) Utils.deserialize(FileUtils.readFileToByteArray(new File(latestPath))); } catch(IOException e) { attempts++; if(attempts >= 10) { throw e; } } } } public Object get(Object key) throws IOException { return snapshot().get(key); } public synchronized void put(Object key, Object val) throws IOException { put(key, val, true); } public synchronized void put(Object key, Object val, boolean cleanup) throws IOException { Map curr = snapshot(); curr.put(key, val); persist(curr, cleanup); } public synchronized void remove(Object key) throws IOException { remove(key, true); } public synchronized void remove(Object key, boolean cleanup) throws IOException { Map curr = snapshot(); curr.remove(key); persist(curr, cleanup); } public synchronized void cleanup(int keepVersions) throws IOException { _vs.cleanup(keepVersions); } private void persist(Map val, boolean cleanup) throws IOException { byte[] toWrite = Utils.serialize(val); String newPath = _vs.createVersion(); FileUtils.writeByteArrayToFile(new File(newPath), toWrite); _vs.succeedVersion(newPath); if(cleanup) _vs.cleanup(4); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/MutableInt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; public class MutableInt { int val; public MutableInt(int val) { this.val = val; } public void set(int val) { this.val = val; } public int get() { return val; } public int increment() { return increment(1); } public int increment(int amt) { val+=amt; return val; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/MutableLong.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; public class MutableLong { long val; public MutableLong(long val) { this.val = val; } public void set(long val) { this.val = val; } public long get() { return val; } public long increment() { return increment(1); } public long increment(long amt) { val+=amt; return val; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/MutableObject.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; public class MutableObject { Object o = null; public MutableObject() { } public MutableObject(Object o) { this.o = o; } public void setObject(Object o) { this.o = o; } public Object getObject() { return o; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/NimbusClient.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import backtype.storm.Config; import backtype.storm.security.auth.ThriftClient; import backtype.storm.generated.Nimbus; import java.util.Map; import org.apache.thrift7.transport.TTransportException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class NimbusClient extends ThriftClient { private Nimbus.Client _client; private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class); public static NimbusClient getConfiguredClient(Map conf) { try { String nimbusHost = (String) conf.get(Config.NIMBUS_HOST); int nimbusPort = Utils.getInt(conf.get(Config.NIMBUS_THRIFT_PORT)); return new NimbusClient(conf, nimbusHost, nimbusPort); } catch (TTransportException ex) { throw new RuntimeException(ex); } } public NimbusClient(Map conf, String host, int port) throws TTransportException { this(conf, host, port, null); } public NimbusClient(Map conf, String host, int port, Integer timeout) throws TTransportException { super(conf, host, port, timeout); _client = new Nimbus.Client(_protocol); } public Nimbus.Client getClient() { return _client; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/RegisteredGlobalState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.HashMap; import java.util.UUID; /** * This class is used as part of testing Storm. It is used to keep track of "global metrics" * in an atomic way. For example, it is used for doing fine-grained detection of when a * local Storm cluster is idle by tracking the number of transferred tuples vs the number of processed * tuples. */ public class RegisteredGlobalState { private static HashMap _states = new HashMap(); private static final Object _lock = new Object(); public static Object globalLock() { return _lock; } public static String registerState(Object init) { synchronized(_lock) { String id = UUID.randomUUID().toString(); _states.put(id, init); return id; } } public static void setState(String id, Object init) { synchronized(_lock) { _states.put(id, init); } } public static Object getState(String id) { synchronized(_lock) { Object ret = _states.get(id); //System.out.println("State: " + ret.toString()); return ret; } } public static void clearState(String id) { synchronized(_lock) { _states.remove(id); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/RotatingMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.Map; import java.util.Map.Entry; /** * Expires keys that have not been updated in the configured number of seconds. * The algorithm used will take between expirationSecs and * expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message. * * get, put, remove, containsKey, and size take O(numBuckets) time to run. * * The advantage of this design is that the expiration thread only locks the object * for O(1) time, meaning the object is essentially always available for gets/puts. */ public class RotatingMap { //this default ensures things expire at most 50% past the expiration time private static final int DEFAULT_NUM_BUCKETS = 3; public static interface ExpiredCallback { public void expire(K key, V val); } private LinkedList> _buckets; private ExpiredCallback _callback; public RotatingMap(int numBuckets, ExpiredCallback callback) { if(numBuckets<2) { throw new IllegalArgumentException("numBuckets must be >= 2"); } _buckets = new LinkedList>(); for(int i=0; i()); } _callback = callback; } public RotatingMap(ExpiredCallback callback) { this(DEFAULT_NUM_BUCKETS, callback); } public RotatingMap(int numBuckets) { this(numBuckets, null); } public Map rotate() { Map dead = _buckets.removeLast(); _buckets.addFirst(new HashMap()); if(_callback!=null) { for(Entry entry: dead.entrySet()) { _callback.expire(entry.getKey(), entry.getValue()); } } return dead; } public boolean containsKey(K key) { for(HashMap bucket: _buckets) { if(bucket.containsKey(key)) { return true; } } return false; } public V get(K key) { for(HashMap bucket: _buckets) { if(bucket.containsKey(key)) { return bucket.get(key); } } return null; } public void put(K key, V value) { Iterator> it = _buckets.iterator(); HashMap bucket = it.next(); bucket.put(key, value); while(it.hasNext()) { bucket = it.next(); bucket.remove(key); } } public Object remove(K key) { for(HashMap bucket: _buckets) { if(bucket.containsKey(key)) { return bucket.remove(key); } } return null; } public int size() { int size = 0; for(HashMap bucket: _buckets) { size+=bucket.size(); } return size; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/ServiceRegistry.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.HashMap; import java.util.UUID; // this class should be combined with RegisteredGlobalState public class ServiceRegistry { private static HashMap _services = new HashMap(); private static final Object _lock = new Object(); public static String registerService(Object service) { synchronized(_lock) { String id = UUID.randomUUID().toString(); _services.put(id, service); return id; } } public static Object getService(String id) { synchronized(_lock) { return _services.get(id); } } public static void unregisterService(String id) { synchronized(_lock) { _services.remove(id); } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/ShellProcess.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import backtype.storm.task.TopologyContext; import java.io.BufferedReader; import java.io.InputStream; import java.io.InputStreamReader; import java.io.DataOutputStream; import java.io.File; import java.io.IOException; import java.util.Map; import java.util.List; import org.apache.commons.io.IOUtils; import org.json.simple.JSONObject; import org.json.simple.JSONValue; import org.apache.log4j.Logger; public class ShellProcess { public static Logger LOG = Logger.getLogger(ShellProcess.class); private DataOutputStream processIn; private BufferedReader processOut; private InputStream processErrorStream; private Process _subprocess; private String[] command; public ShellProcess(String[] command) { this.command = command; } public Number launch(Map conf, TopologyContext context) throws IOException { ProcessBuilder builder = new ProcessBuilder(command); builder.directory(new File(context.getCodeDir())); _subprocess = builder.start(); processIn = new DataOutputStream(_subprocess.getOutputStream()); processOut = new BufferedReader(new InputStreamReader(_subprocess.getInputStream())); processErrorStream = _subprocess.getErrorStream(); JSONObject setupInfo = new JSONObject(); setupInfo.put("pidDir", context.getPIDDir()); setupInfo.put("conf", conf); setupInfo.put("context", context); writeMessage(setupInfo); return (Number)readMessage().get("pid"); } public void destroy() { _subprocess.destroy(); } public void writeMessage(Object msg) throws IOException { writeString(JSONValue.toJSONString(msg)); } private void writeString(String str) throws IOException { byte[] strBytes = str.getBytes("UTF-8"); processIn.write(strBytes, 0, strBytes.length); processIn.writeBytes("\nend\n"); processIn.flush(); } public JSONObject readMessage() throws IOException { String string = readString(); JSONObject msg = (JSONObject)JSONValue.parse(string); if (msg != null) { return msg; } else { throw new IOException("unable to parse: " + string); } } public String getErrorsString() { if(processErrorStream!=null) { try { return IOUtils.toString(processErrorStream); } catch(IOException e) { return "(Unable to capture error stream)"; } } else { return ""; } } public void drainErrorStream() { try { while (processErrorStream.available() > 0) { int bufferSize = processErrorStream.available(); byte[] errorReadingBuffer = new byte[bufferSize]; processErrorStream.read(errorReadingBuffer, 0, bufferSize); LOG.info("Got error from shell process: " + new String(errorReadingBuffer)); } } catch(Exception e) { } } private String readString() throws IOException { StringBuilder line = new StringBuilder(); //synchronized (processOut) { while (true) { String subline = processOut.readLine(); if(subline==null) { StringBuilder errorMessage = new StringBuilder(); errorMessage.append("Pipe to subprocess seems to be broken!"); if (line.length() == 0) { errorMessage.append(" No output read.\n"); } else { errorMessage.append(" Currently read output: " + line.toString() + "\n"); } errorMessage.append("Shell Process Exception:\n"); errorMessage.append(getErrorsString() + "\n"); throw new RuntimeException(errorMessage.toString()); } if(subline.equals("end")) { break; } if(line.length()!=0) { line.append("\n"); } line.append(subline); } //} return line.toString(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/ThriftTopologyUtils.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import backtype.storm.generated.Bolt; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.SpoutSpec; import backtype.storm.generated.StateSpoutSpec; import backtype.storm.generated.StormTopology; import java.util.HashSet; import java.util.Map; import java.util.Set; public class ThriftTopologyUtils { public static Set getComponentIds(StormTopology topology) { Set ret = new HashSet(); for(StormTopology._Fields f: StormTopology.metaDataMap.keySet()) { Map componentMap = (Map) topology.getFieldValue(f); ret.addAll(componentMap.keySet()); } return ret; } public static ComponentCommon getComponentCommon(StormTopology topology, String componentId) { for(StormTopology._Fields f: StormTopology.metaDataMap.keySet()) { Map componentMap = (Map) topology.getFieldValue(f); if(componentMap.containsKey(componentId)) { Object component = componentMap.get(componentId); if(component instanceof Bolt) { return ((Bolt) component).get_common(); } if(component instanceof SpoutSpec) { return ((SpoutSpec) component).get_common(); } if(component instanceof StateSpoutSpec) { return ((StateSpoutSpec) component).get_common(); } throw new RuntimeException("Unreachable code! No get_common conversion for component " + component); } } throw new IllegalArgumentException("Could not find component common for " + componentId); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/Time.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.concurrent.atomic.AtomicLong; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class Time { public static Logger LOG = LoggerFactory.getLogger(Time.class); private static AtomicBoolean simulating = new AtomicBoolean(false); //TODO: should probably use weak references here or something private static volatile Map threadSleepTimes; private static final Object sleepTimesLock = new Object(); private static AtomicLong simulatedCurrTimeMs; //should this be a thread local that's allowed to keep advancing? public static void startSimulating() { simulating.set(true); simulatedCurrTimeMs = new AtomicLong(0); threadSleepTimes = new ConcurrentHashMap(); } public static void stopSimulating() { simulating.set(false); threadSleepTimes = null; } public static boolean isSimulating() { return simulating.get(); } public static void sleepUntil(long targetTimeMs) throws InterruptedException { if(simulating.get()) { try { synchronized(sleepTimesLock) { threadSleepTimes.put(Thread.currentThread(), new AtomicLong(targetTimeMs)); } while(simulatedCurrTimeMs.get() < targetTimeMs) { Thread.sleep(10); } } finally { synchronized(sleepTimesLock) { threadSleepTimes.remove(Thread.currentThread()); } } } else { long sleepTime = targetTimeMs-currentTimeMillis(); if(sleepTime>0) Thread.sleep(sleepTime); } } public static void sleep(long ms) throws InterruptedException { sleepUntil(currentTimeMillis()+ms); } public static long currentTimeMillis() { if(simulating.get()) { return simulatedCurrTimeMs.get(); } else { return System.currentTimeMillis(); } } public static int currentTimeSecs() { return (int) (currentTimeMillis() / 1000); } public static void advanceTime(long ms) { if(!simulating.get()) throw new IllegalStateException("Cannot simulate time unless in simulation mode"); simulatedCurrTimeMs.set(simulatedCurrTimeMs.get() + ms); } public static boolean isThreadWaiting(Thread t) { if(!simulating.get()) throw new IllegalStateException("Must be in simulation mode"); AtomicLong time; synchronized(sleepTimesLock) { time = threadSleepTimes.get(t); } return !t.isAlive() || time!=null && currentTimeMillis() < time.longValue(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/TimeCacheMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.Map; import java.util.Map.Entry; /** * Expires keys that have not been updated in the configured number of seconds. * The algorithm used will take between expirationSecs and * expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message. * * get, put, remove, containsKey, and size take O(numBuckets) time to run. * * The advantage of this design is that the expiration thread only locks the object * for O(1) time, meaning the object is essentially always available for gets/puts. */ //deprecated in favor of non-threaded RotatingMap @Deprecated public class TimeCacheMap { //this default ensures things expire at most 50% past the expiration time private static final int DEFAULT_NUM_BUCKETS = 3; public static interface ExpiredCallback { public void expire(K key, V val); } private LinkedList> _buckets; private final Object _lock = new Object(); private Thread _cleaner; private ExpiredCallback _callback; public TimeCacheMap(int expirationSecs, int numBuckets, ExpiredCallback callback) { if(numBuckets<2) { throw new IllegalArgumentException("numBuckets must be >= 2"); } _buckets = new LinkedList>(); for(int i=0; i()); } _callback = callback; final long expirationMillis = expirationSecs * 1000L; final long sleepTime = expirationMillis / (numBuckets-1); _cleaner = new Thread(new Runnable() { public void run() { try { while(true) { Map dead = null; Time.sleep(sleepTime); synchronized(_lock) { dead = _buckets.removeLast(); _buckets.addFirst(new HashMap()); } if(_callback!=null) { for(Entry entry: dead.entrySet()) { _callback.expire(entry.getKey(), entry.getValue()); } } } } catch (InterruptedException ex) { } } }); _cleaner.setDaemon(true); _cleaner.start(); } public TimeCacheMap(int expirationSecs, ExpiredCallback callback) { this(expirationSecs, DEFAULT_NUM_BUCKETS, callback); } public TimeCacheMap(int expirationSecs) { this(expirationSecs, DEFAULT_NUM_BUCKETS); } public TimeCacheMap(int expirationSecs, int numBuckets) { this(expirationSecs, numBuckets, null); } public boolean containsKey(K key) { synchronized(_lock) { for(HashMap bucket: _buckets) { if(bucket.containsKey(key)) { return true; } } return false; } } public V get(K key) { synchronized(_lock) { for(HashMap bucket: _buckets) { if(bucket.containsKey(key)) { return bucket.get(key); } } return null; } } public void put(K key, V value) { synchronized(_lock) { Iterator> it = _buckets.iterator(); HashMap bucket = it.next(); bucket.put(key, value); while(it.hasNext()) { bucket = it.next(); bucket.remove(key); } } } public Object remove(K key) { synchronized(_lock) { for(HashMap bucket: _buckets) { if(bucket.containsKey(key)) { return bucket.remove(key); } } return null; } } public int size() { synchronized(_lock) { int size = 0; for(HashMap bucket: _buckets) { size+=bucket.size(); } return size; } } public void cleanup() { _cleaner.interrupt(); } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/Utils.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import backtype.storm.Config; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.ComponentObject; import backtype.storm.generated.StormTopology; import clojure.lang.IFn; import clojure.lang.RT; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.CuratorFrameworkFactory; import com.netflix.curator.retry.ExponentialBackoffRetry; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.FileOutputStream; import java.io.InputStreamReader; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.net.URL; import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; import java.util.ArrayList; import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.TreeMap; import java.util.UUID; import org.apache.commons.lang.StringUtils; import org.apache.thrift7.TException; import org.json.simple.JSONValue; import org.yaml.snakeyaml.Yaml; public class Utils { public static final String DEFAULT_STREAM_ID = "default"; public static Object newInstance(String klass) { try { Class c = Class.forName(klass); return c.newInstance(); } catch (Exception e) { throw new RuntimeException(e); } } public static byte[] serialize(Object obj) { try { ByteArrayOutputStream bos = new ByteArrayOutputStream(); ObjectOutputStream oos = new ObjectOutputStream(bos); oos.writeObject(obj); oos.close(); return bos.toByteArray(); } catch(IOException ioe) { throw new RuntimeException(ioe); } } public static Object deserialize(byte[] serialized) { try { ByteArrayInputStream bis = new ByteArrayInputStream(serialized); ObjectInputStream ois = new ObjectInputStream(bis); Object ret = ois.readObject(); ois.close(); return ret; } catch(IOException ioe) { throw new RuntimeException(ioe); } catch(ClassNotFoundException e) { throw new RuntimeException(e); } } public static String join(Iterable coll, String sep) { Iterator it = coll.iterator(); String ret = ""; while(it.hasNext()) { ret = ret + it.next(); if(it.hasNext()) { ret = ret + sep; } } return ret; } public static void sleep(long millis) { try { Time.sleep(millis); } catch(InterruptedException e) { throw new RuntimeException(e); } } public static List findResources(String name) { try { Enumeration resources = Thread.currentThread().getContextClassLoader().getResources(name); List ret = new ArrayList(); while(resources.hasMoreElements()) { ret.add(resources.nextElement()); } return ret; } catch(IOException e) { throw new RuntimeException(e); } } public static Map findAndReadConfigFile(String name, boolean mustExist) { try { HashSet resources = new HashSet(findResources(name)); if(resources.isEmpty()) { if(mustExist) throw new RuntimeException("Could not find config file on classpath " + name); else return new HashMap(); } if(resources.size() > 1) { throw new RuntimeException("Found multiple " + name + " resources. You're probably bundling the Storm jars with your topology jar. " + resources); } URL resource = resources.iterator().next(); Yaml yaml = new Yaml(); Map ret = (Map) yaml.load(new InputStreamReader(resource.openStream())); if(ret==null) ret = new HashMap(); return new HashMap(ret); } catch (IOException e) { throw new RuntimeException(e); } } public static Map findAndReadConfigFile(String name) { return findAndReadConfigFile(name, true); } public static Map readDefaultConfig() { return findAndReadConfigFile("defaults.yaml", true); } public static Map readCommandLineOpts() { Map ret = new HashMap(); String commandOptions = System.getProperty("storm.options"); if(commandOptions != null) { commandOptions = commandOptions.replaceAll("%%%%", " "); String[] configs = commandOptions.split(","); for (String config : configs) { String[] options = config.split("="); if (options.length == 2) { ret.put(options[0], options[1]); } } } return ret; } public static Map readStormConfig() { Map ret = readDefaultConfig(); String confFile = System.getProperty("storm.conf.file"); Map storm; if (confFile==null || confFile.equals("")) { storm = findAndReadConfigFile("storm.yaml", false); } else { storm = findAndReadConfigFile(confFile, true); } ret.putAll(storm); ret.putAll(readCommandLineOpts()); return ret; } private static Object normalizeConf(Object conf) { if(conf==null) return new HashMap(); if(conf instanceof Map) { Map confMap = new HashMap((Map) conf); for(Object key: confMap.keySet()) { Object val = confMap.get(key); confMap.put(key, normalizeConf(val)); } return confMap; } else if(conf instanceof List) { List confList = new ArrayList((List) conf); for(int i=0; i stormConf) { return normalizeConf(stormConf).equals(normalizeConf((Map) JSONValue.parse(JSONValue.toJSONString(stormConf)))); } public static Object getSetComponentObject(ComponentObject obj) { if(obj.getSetField()==ComponentObject._Fields.SERIALIZED_JAVA) { return Utils.deserialize(obj.get_serialized_java()); } else if(obj.getSetField()==ComponentObject._Fields.JAVA_OBJECT) { return obj.get_java_object(); } else { return obj.get_shell(); } } public static T get(Map m, S key, T def) { T ret = m.get(key); if(ret==null) { ret = def; } return ret; } public static List tuple(Object... values) { List ret = new ArrayList(); for(Object v: values) { ret.add(v); } return ret; } public static void downloadFromMaster(Map conf, String file, String localFile) throws IOException, TException { NimbusClient client = NimbusClient.getConfiguredClient(conf); String id = client.getClient().beginFileDownload(file); WritableByteChannel out = Channels.newChannel(new FileOutputStream(localFile)); while(true) { ByteBuffer chunk = client.getClient().downloadChunk(id); int written = out.write(chunk); if(written==0) break; } out.close(); } public static IFn loadClojureFn(String namespace, String name) { try { clojure.lang.Compiler.eval(RT.readString("(require '" + namespace + ")")); } catch (Exception e) { //if playing from the repl and defining functions, file won't exist } return (IFn) RT.var(namespace, name).deref(); } public static boolean isSystemId(String id) { return id.startsWith("__"); } public static Map reverseMap(Map map) { Map ret = new HashMap(); for(K key: map.keySet()) { ret.put(map.get(key), key); } return ret; } public static ComponentCommon getComponentCommon(StormTopology topology, String id) { if(topology.get_spouts().containsKey(id)) { return topology.get_spouts().get(id).get_common(); } if(topology.get_bolts().containsKey(id)) { return topology.get_bolts().get(id).get_common(); } if(topology.get_state_spouts().containsKey(id)) { return topology.get_state_spouts().get(id).get_common(); } throw new IllegalArgumentException("Could not find component with id " + id); } public static Integer getInt(Object o) { if(o instanceof Long) { return ((Long) o ).intValue(); } else if (o instanceof Integer) { return (Integer) o; } else if (o instanceof Short) { return ((Short) o).intValue(); } else { throw new IllegalArgumentException("Don't know how to convert " + o + " + to int"); } } public static long secureRandomLong() { return UUID.randomUUID().getLeastSignificantBits(); } public static CuratorFramework newCurator(Map conf, List servers, Object port, String root) { return newCurator(conf, servers, port, root, null); } public static class BoundedExponentialBackoffRetry extends ExponentialBackoffRetry { protected final int maxRetryInterval; public BoundedExponentialBackoffRetry(int baseSleepTimeMs, int maxRetries, int maxSleepTimeMs) { super(baseSleepTimeMs, maxRetries); this.maxRetryInterval = maxSleepTimeMs; } public int getMaxRetryInterval() { return this.maxRetryInterval; } @Override public int getSleepTimeMs(int count, long elapsedMs) { return Math.min(maxRetryInterval, super.getSleepTimeMs(count, elapsedMs)); } } public static CuratorFramework newCurator(Map conf, List servers, Object port, String root, ZookeeperAuthInfo auth) { List serverPorts = new ArrayList(); for(String zkServer: (List) servers) { serverPorts.add(zkServer + ":" + Utils.getInt(port)); } String zkStr = StringUtils.join(serverPorts, ",") + root; try { CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder() .connectString(zkStr) .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT))) .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT))) .retryPolicy(new BoundedExponentialBackoffRetry( Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)), Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)), Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)))); if(auth!=null && auth.scheme!=null) { builder = builder.authorization(auth.scheme, auth.payload); } return builder.build(); } catch (IOException e) { throw new RuntimeException(e); } } public static CuratorFramework newCurator(Map conf, List servers, Object port) { return newCurator(conf, servers, port, ""); } public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port, String root) { CuratorFramework ret = newCurator(conf, servers, port, root); ret.start(); return ret; } public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port) { CuratorFramework ret = newCurator(conf, servers, port); ret.start(); return ret; } /** * (defn integer-divided [sum num-pieces] (let [base (int (/ sum num-pieces)) num-inc (mod sum num-pieces) num-bases (- num-pieces num-inc)] (if (= num-inc 0) {base num-bases} {base num-bases (inc base) num-inc} ))) * @param sum * @param numPieces * @return */ public static TreeMap integerDivided(int sum, int numPieces) { int base = sum / numPieces; int numInc = sum % numPieces; int numBases = numPieces - numInc; TreeMap ret = new TreeMap(); ret.put(base, numBases); if(numInc!=0) { ret.put(base+1, numInc); } return ret; } public static byte[] toByteArray(ByteBuffer buffer) { byte[] ret = new byte[buffer.remaining()]; buffer.get(ret, 0, ret.length); return ret; } public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) { Throwable t = throwable; while(t != null) { if(klass.isInstance(t)) { return true; } t = t.getCause(); } return false; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/VersionedStore.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.io.File; import org.apache.commons.io.FileUtils; public class VersionedStore { private static final String FINISHED_VERSION_SUFFIX = ".version"; private String _root; public VersionedStore(String path) throws IOException { _root = path; mkdirs(_root); } public String getRoot() { return _root; } public String versionPath(long version) { return new File(_root, "" + version).getAbsolutePath(); } public String mostRecentVersionPath() throws IOException { Long v = mostRecentVersion(); if(v==null) return null; return versionPath(v); } public String mostRecentVersionPath(long maxVersion) throws IOException { Long v = mostRecentVersion(maxVersion); if(v==null) return null; return versionPath(v); } public Long mostRecentVersion() throws IOException { List all = getAllVersions(); if(all.size()==0) return null; return all.get(0); } public Long mostRecentVersion(long maxVersion) throws IOException { List all = getAllVersions(); for(Long v: all) { if(v <= maxVersion) return v; } return null; } public String createVersion() throws IOException { Long mostRecent = mostRecentVersion(); long version = Time.currentTimeMillis(); if(mostRecent!=null && version <= mostRecent) { version = mostRecent + 1; } return createVersion(version); } public String createVersion(long version) throws IOException { String ret = versionPath(version); if(getAllVersions().contains(version)) throw new RuntimeException("Version already exists or data already exists"); else return ret; } public void failVersion(String path) throws IOException { deleteVersion(validateAndGetVersion(path)); } public void deleteVersion(long version) throws IOException { File versionFile = new File(versionPath(version)); File tokenFile = new File(tokenPath(version)); if(versionFile.exists()) { FileUtils.forceDelete(versionFile); } if(tokenFile.exists()) { FileUtils.forceDelete(tokenFile); } } public void succeedVersion(String path) throws IOException { long version = validateAndGetVersion(path); // should rewrite this to do a file move createNewFile(tokenPath(version)); } public void cleanup() throws IOException { cleanup(-1); } public void cleanup(int versionsToKeep) throws IOException { List versions = getAllVersions(); if(versionsToKeep >= 0) { versions = versions.subList(0, Math.min(versions.size(), versionsToKeep)); } HashSet keepers = new HashSet(versions); for(String p: listDir(_root)) { Long v = parseVersion(p); if(v!=null && !keepers.contains(v)) { deleteVersion(v); } } } /** * Sorted from most recent to oldest */ public List getAllVersions() throws IOException { List ret = new ArrayList(); for(String s: listDir(_root)) { if(s.endsWith(FINISHED_VERSION_SUFFIX)) { ret.add(validateAndGetVersion(s)); } } Collections.sort(ret); Collections.reverse(ret); return ret; } private String tokenPath(long version) { return new File(_root, "" + version + FINISHED_VERSION_SUFFIX).getAbsolutePath(); } private long validateAndGetVersion(String path) { Long v = parseVersion(path); if(v==null) throw new RuntimeException(path + " is not a valid version"); return v; } private Long parseVersion(String path) { String name = new File(path).getName(); if(name.endsWith(FINISHED_VERSION_SUFFIX)) { name = name.substring(0, name.length()-FINISHED_VERSION_SUFFIX.length()); } try { return Long.parseLong(name); } catch(NumberFormatException e) { return null; } } private void createNewFile(String path) throws IOException { new File(path).createNewFile(); } private void mkdirs(String path) throws IOException { new File(path).mkdirs(); } private List listDir(String dir) throws IOException { List ret = new ArrayList(); File[] contents = new File(dir).listFiles(); if(contents!=null) { for(File f: contents) { ret.add(f.getAbsolutePath()); } } return ret; } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/WindowedTimeThrottler.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.utils; public class WindowedTimeThrottler { long _windowMillis; int _maxAmt; long _windowStartTime; int _windowEvents = 0; public WindowedTimeThrottler(Number windowMillis, Number maxAmt) { _windowMillis = windowMillis.longValue(); _maxAmt = maxAmt.intValue(); _windowStartTime = System.currentTimeMillis(); } public boolean isThrottled() { resetIfNecessary(); return _windowEvents >= _maxAmt; } //returns void if the event should continue, false if the event should not be done public void markEvent() { resetIfNecessary(); _windowEvents++; } private void resetIfNecessary() { long now = System.currentTimeMillis(); if(now - _windowStartTime > _windowMillis) { _windowStartTime = now; _windowEvents = 0; } } } ================================================ FILE: storm-core/src/jvm/backtype/storm/utils/WritableUtils.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT 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 file originally comes from the Apache Hadoop project. Changes have been made to the file. * */ package backtype.storm.utils; import java.io.*; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; public final class WritableUtils { public static byte[] readCompressedByteArray(DataInput in) throws IOException { int length = in.readInt(); if (length == -1) return null; byte[] buffer = new byte[length]; in.readFully(buffer); // could/should use readFully(buffer,0,length)? GZIPInputStream gzi = new GZIPInputStream(new ByteArrayInputStream(buffer, 0, buffer.length)); byte[] outbuf = new byte[length]; ByteArrayOutputStream bos = new ByteArrayOutputStream(); int len; while((len=gzi.read(outbuf, 0, outbuf.length)) != -1){ bos.write(outbuf, 0, len); } byte[] decompressed = bos.toByteArray(); bos.close(); gzi.close(); return decompressed; } public static void skipCompressedByteArray(DataInput in) throws IOException { int length = in.readInt(); if (length != -1) { skipFully(in, length); } } public static int writeCompressedByteArray(DataOutput out, byte[] bytes) throws IOException { if (bytes != null) { ByteArrayOutputStream bos = new ByteArrayOutputStream(); GZIPOutputStream gzout = new GZIPOutputStream(bos); gzout.write(bytes, 0, bytes.length); gzout.close(); byte[] buffer = bos.toByteArray(); int len = buffer.length; out.writeInt(len); out.write(buffer, 0, len); /* debug only! Once we have confidence, can lose this. */ return ((bytes.length != 0) ? (100*buffer.length)/bytes.length : 0); } else { out.writeInt(-1); return -1; } } /* Ugly utility, maybe someone else can do this better */ public static String readCompressedString(DataInput in) throws IOException { byte[] bytes = readCompressedByteArray(in); if (bytes == null) return null; return new String(bytes, "UTF-8"); } public static int writeCompressedString(DataOutput out, String s) throws IOException { return writeCompressedByteArray(out, (s != null) ? s.getBytes("UTF-8") : null); } /* * * Write a String as a Network Int n, followed by n Bytes * Alternative to 16 bit read/writeUTF. * Encoding standard is... ? * */ public static void writeString(DataOutput out, String s) throws IOException { if (s != null) { byte[] buffer = s.getBytes("UTF-8"); int len = buffer.length; out.writeInt(len); out.write(buffer, 0, len); } else { out.writeInt(-1); } } /* * Read a String as a Network Int n, followed by n Bytes * Alternative to 16 bit read/writeUTF. * Encoding standard is... ? * */ public static String readString(DataInput in) throws IOException{ int length = in.readInt(); if (length == -1) return null; byte[] buffer = new byte[length]; in.readFully(buffer); // could/should use readFully(buffer,0,length)? return new String(buffer,"UTF-8"); } /* * Write a String array as a Nework Int N, followed by Int N Byte Array Strings. * Could be generalised using introspection. * */ public static void writeStringArray(DataOutput out, String[] s) throws IOException{ out.writeInt(s.length); for(int i = 0; i < s.length; i++) { writeString(out, s[i]); } } /* * Write a String array as a Nework Int N, followed by Int N Byte Array of * compressed Strings. Handles also null arrays and null values. * Could be generalised using introspection. * */ public static void writeCompressedStringArray(DataOutput out, String[] s) throws IOException{ if (s == null) { out.writeInt(-1); return; } out.writeInt(s.length); for(int i = 0; i < s.length; i++) { writeCompressedString(out, s[i]); } } /* * Write a String array as a Nework Int N, followed by Int N Byte Array Strings. * Could be generalised using introspection. Actually this bit couldn't... * */ public static String[] readStringArray(DataInput in) throws IOException { int len = in.readInt(); if (len == -1) return null; String[] s = new String[len]; for(int i = 0; i < len; i++) { s[i] = readString(in); } return s; } /* * Write a String array as a Nework Int N, followed by Int N Byte Array Strings. * Could be generalised using introspection. Handles null arrays and null values. * */ public static String[] readCompressedStringArray(DataInput in) throws IOException { int len = in.readInt(); if (len == -1) return null; String[] s = new String[len]; for(int i = 0; i < len; i++) { s[i] = readCompressedString(in); } return s; } /* * * Test Utility Method Display Byte Array. * */ public static void displayByteArray(byte[] record){ int i; for(i=0;i < record.length -1; i++){ if (i % 16 == 0) { System.out.println(); } System.out.print(Integer.toHexString(record[i] >> 4 & 0x0F)); System.out.print(Integer.toHexString(record[i] & 0x0F)); System.out.print(","); } System.out.print(Integer.toHexString(record[i] >> 4 & 0x0F)); System.out.print(Integer.toHexString(record[i] & 0x0F)); System.out.println(); } /** * Serializes an integer to a binary stream with zero-compressed encoding. * For -120 <= i <= 127, only one byte is used with the actual value. * For other values of i, the first byte value indicates whether the * integer is positive or negative, and the number of bytes that follow. * If the first byte value v is between -121 and -124, the following integer * is positive, with number of bytes that follow are -(v+120). * If the first byte value v is between -125 and -128, the following integer * is negative, with number of bytes that follow are -(v+124). Bytes are * stored in the high-non-zero-byte-first order. * * @param stream Binary output stream * @param i Integer to be serialized * @throws java.io.IOException */ public static void writeVInt(DataOutput stream, int i) throws IOException { writeVLong(stream, i); } /** * Serializes a long to a binary stream with zero-compressed encoding. * For -112 <= i <= 127, only one byte is used with the actual value. * For other values of i, the first byte value indicates whether the * long is positive or negative, and the number of bytes that follow. * If the first byte value v is between -113 and -120, the following long * is positive, with number of bytes that follow are -(v+112). * If the first byte value v is between -121 and -128, the following long * is negative, with number of bytes that follow are -(v+120). Bytes are * stored in the high-non-zero-byte-first order. * * @param stream Binary output stream * @param i Long to be serialized * @throws java.io.IOException */ public static void writeVLong(DataOutput stream, long i) throws IOException { if (i >= -112 && i <= 127) { stream.writeByte((byte)i); return; } int len = -112; if (i < 0) { i ^= -1L; // take one's complement' len = -120; } long tmp = i; while (tmp != 0) { tmp = tmp >> 8; len--; } stream.writeByte((byte)len); len = (len < -120) ? -(len + 120) : -(len + 112); for (int idx = len; idx != 0; idx--) { int shiftbits = (idx - 1) * 8; long mask = 0xFFL << shiftbits; stream.writeByte((byte)((i & mask) >> shiftbits)); } } /** * Reads a zero-compressed encoded long from input stream and returns it. * @param stream Binary input stream * @throws java.io.IOException * @return deserialized long from stream. */ public static long readVLong(DataInput stream) throws IOException { byte firstByte = stream.readByte(); int len = decodeVIntSize(firstByte); if (len == 1) { return firstByte; } long i = 0; for (int idx = 0; idx < len-1; idx++) { byte b = stream.readByte(); i = i << 8; i = i | (b & 0xFF); } return (isNegativeVInt(firstByte) ? (i ^ -1L) : i); } /** * Reads a zero-compressed encoded integer from input stream and returns it. * @param stream Binary input stream * @throws java.io.IOException * @return deserialized integer from stream. */ public static int readVInt(DataInput stream) throws IOException { return (int) readVLong(stream); } /** * Given the first byte of a vint/vlong, determine the sign * @param value the first byte * @return is the value negative */ public static boolean isNegativeVInt(byte value) { return value < -120 || (value >= -112 && value < 0); } /** * Parse the first byte of a vint/vlong to determine the number of bytes * @param value the first byte of the vint/vlong * @return the total number of bytes (1 to 9) */ public static int decodeVIntSize(byte value) { if (value >= -112) { return 1; } else if (value < -120) { return -119 - value; } return -111 - value; } /** * Get the encoded length if an integer is stored in a variable-length format * @return the encoded length */ public static int getVIntSize(long i) { if (i >= -112 && i <= 127) { return 1; } if (i < 0) { i ^= -1L; // take one's complement' } // find the number of bytes with non-leading zeros int dataBits = Long.SIZE - Long.numberOfLeadingZeros(i); // find the number of data bytes + length byte return (dataBits + 7) / 8 + 1; } /** * Skip len number of bytes in input streamin * @param in input stream * @param len number of bytes to skip * @throws IOException when skipped less number of bytes */ public static void skipFully(DataInput in, int len) throws IOException { int total = 0; int cur = 0; while ((total 0)) { total += cur; } if (total mixed(JoinType... types) { return Arrays.asList(types); } } ================================================ FILE: storm-core/src/jvm/storm/trident/Stream.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident; import backtype.storm.generated.Grouping; import backtype.storm.generated.NullStruct; import storm.trident.fluent.ChainedAggregatorDeclarer; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import storm.trident.fluent.GlobalAggregationScheme; import storm.trident.fluent.GroupedStream; import storm.trident.fluent.IAggregatableStream; import storm.trident.operation.Aggregator; import storm.trident.operation.Assembly; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.Filter; import storm.trident.operation.Function; import storm.trident.operation.ReducerAggregator; import storm.trident.operation.impl.CombinerAggStateUpdater; import storm.trident.operation.impl.FilterExecutor; import storm.trident.operation.impl.GlobalBatchToPartition; import storm.trident.operation.impl.ReducerAggStateUpdater; import storm.trident.operation.impl.IndexHashBatchToPartition; import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; import storm.trident.operation.impl.TrueFilter; import storm.trident.partition.GlobalGrouping; import storm.trident.partition.IdentityGrouping; import storm.trident.partition.IndexHashGrouping; import storm.trident.planner.Node; import storm.trident.planner.NodeStateInfo; import storm.trident.planner.PartitionNode; import storm.trident.planner.ProcessorNode; import storm.trident.planner.processor.AggregateProcessor; import storm.trident.planner.processor.EachProcessor; import storm.trident.planner.processor.PartitionPersistProcessor; import storm.trident.planner.processor.ProjectedProcessor; import storm.trident.planner.processor.StateQueryProcessor; import storm.trident.state.QueryFunction; import storm.trident.state.StateFactory; import storm.trident.state.StateSpec; import storm.trident.state.StateUpdater; import storm.trident.util.TridentUtils; // TODO: need to be able to replace existing fields with the function fields (like Cascading Fields.REPLACE) public class Stream implements IAggregatableStream { Node _node; TridentTopology _topology; String _name; protected Stream(TridentTopology topology, String name, Node node) { _topology = topology; _node = node; _name = name; } public Stream name(String name) { return new Stream(_topology, name, _node); } public Stream parallelismHint(int hint) { _node.parallelismHint = hint; return this; } public Stream project(Fields keepFields) { projectionValidation(keepFields); return _topology.addSourcedNode(this, new ProcessorNode(_topology.getUniqueStreamId(), _name, keepFields, new Fields(), new ProjectedProcessor(keepFields))); } public GroupedStream groupBy(Fields fields) { projectionValidation(fields); return new GroupedStream(this, fields); } public Stream partitionBy(Fields fields) { projectionValidation(fields); return partition(Grouping.fields(fields.toList())); } public Stream partition(CustomStreamGrouping partitioner) { return partition(Grouping.custom_serialized(Utils.serialize(partitioner))); } public Stream shuffle() { return partition(Grouping.shuffle(new NullStruct())); } public Stream global() { // use this instead of storm's built in one so that we can specify a singleemitbatchtopartition // without knowledge of storm's internals return partition(new GlobalGrouping()); } public Stream batchGlobal() { // the first field is the batch id return partition(new IndexHashGrouping(0)); } public Stream broadcast() { return partition(Grouping.all(new NullStruct())); } public Stream identityPartition() { return partition(new IdentityGrouping()); } public Stream partition(Grouping grouping) { if(_node instanceof PartitionNode) { return each(new Fields(), new TrueFilter()).partition(grouping); } else { return _topology.addSourcedNode(this, new PartitionNode(_node.streamId, _name, getOutputFields(), grouping)); } } public Stream applyAssembly(Assembly assembly) { return assembly.apply(this); } @Override public Stream each(Fields inputFields, Function function, Fields functionFields) { projectionValidation(inputFields); return _topology.addSourcedNode(this, new ProcessorNode(_topology.getUniqueStreamId(), _name, TridentUtils.fieldsConcat(getOutputFields(), functionFields), functionFields, new EachProcessor(inputFields, function))); } //creates brand new tuples with brand new fields @Override public Stream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields) { projectionValidation(inputFields); return _topology.addSourcedNode(this, new ProcessorNode(_topology.getUniqueStreamId(), _name, functionFields, functionFields, new AggregateProcessor(inputFields, agg))); } public Stream stateQuery(TridentState state, Fields inputFields, QueryFunction function, Fields functionFields) { projectionValidation(inputFields); String stateId = state._node.stateInfo.id; Node n = new ProcessorNode(_topology.getUniqueStreamId(), _name, TridentUtils.fieldsConcat(getOutputFields(), functionFields), functionFields, new StateQueryProcessor(stateId, inputFields, function)); _topology._colocate.get(stateId).add(n); return _topology.addSourcedNode(this, n); } public TridentState partitionPersist(StateFactory stateFactory, Fields inputFields, StateUpdater updater, Fields functionFields) { return partitionPersist(new StateSpec(stateFactory), inputFields, updater, functionFields); } public TridentState partitionPersist(StateSpec stateSpec, Fields inputFields, StateUpdater updater, Fields functionFields) { projectionValidation(inputFields); String id = _topology.getUniqueStateId(); ProcessorNode n = new ProcessorNode(_topology.getUniqueStreamId(), _name, functionFields, functionFields, new PartitionPersistProcessor(id, inputFields, updater)); n.committer = true; n.stateInfo = new NodeStateInfo(id, stateSpec); return _topology.addSourcedStateNode(this, n); } public TridentState partitionPersist(StateFactory stateFactory, Fields inputFields, StateUpdater updater) { return partitionPersist(stateFactory, inputFields, updater, new Fields()); } public TridentState partitionPersist(StateSpec stateSpec, Fields inputFields, StateUpdater updater) { return partitionPersist(stateSpec, inputFields, updater, new Fields()); } public Stream each(Function function, Fields functionFields) { return each(null, function, functionFields); } public Stream each(Fields inputFields, Filter filter) { return each(inputFields, new FilterExecutor(filter), new Fields()); } public ChainedAggregatorDeclarer chainedAgg() { return new ChainedAggregatorDeclarer(this, new BatchGlobalAggScheme()); } public Stream partitionAggregate(Aggregator agg, Fields functionFields) { return partitionAggregate(null, agg, functionFields); } public Stream partitionAggregate(CombinerAggregator agg, Fields functionFields) { return partitionAggregate(null, agg, functionFields); } public Stream partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields) { projectionValidation(inputFields); return chainedAgg() .partitionAggregate(inputFields, agg, functionFields) .chainEnd(); } public Stream partitionAggregate(ReducerAggregator agg, Fields functionFields) { return partitionAggregate(null, agg, functionFields); } public Stream partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields) { projectionValidation(inputFields); return chainedAgg() .partitionAggregate(inputFields, agg, functionFields) .chainEnd(); } public Stream aggregate(Aggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public Stream aggregate(Fields inputFields, Aggregator agg, Fields functionFields) { projectionValidation(inputFields); return chainedAgg() .aggregate(inputFields, agg, functionFields) .chainEnd(); } public Stream aggregate(CombinerAggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public Stream aggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields) { projectionValidation(inputFields); return chainedAgg() .aggregate(inputFields, agg, functionFields) .chainEnd(); } public Stream aggregate(ReducerAggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public Stream aggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields) { projectionValidation(inputFields); return chainedAgg() .aggregate(inputFields, agg, functionFields) .chainEnd(); } public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater, Fields functionFields) { return partitionPersist(new StateSpec(stateFactory), updater, functionFields); } public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater, Fields functionFields) { return partitionPersist(stateSpec, null, updater, functionFields); } public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater) { return partitionPersist(stateFactory, updater, new Fields()); } public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater) { return partitionPersist(stateSpec, updater, new Fields()); } public TridentState persistentAggregate(StateFactory stateFactory, CombinerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, CombinerAggregator agg, Fields functionFields) { return persistentAggregate(spec, null, agg, functionFields); } public TridentState persistentAggregate(StateFactory stateFactory, Fields inputFields, CombinerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), inputFields, agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, Fields inputFields, CombinerAggregator agg, Fields functionFields) { projectionValidation(inputFields); // replaces normal aggregation here with a global grouping because it needs to be consistent across batches return new ChainedAggregatorDeclarer(this, new GlobalAggScheme()) .aggregate(inputFields, agg, functionFields) .chainEnd() .partitionPersist(spec, functionFields, new CombinerAggStateUpdater(agg), functionFields); } public TridentState persistentAggregate(StateFactory stateFactory, ReducerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, ReducerAggregator agg, Fields functionFields) { return persistentAggregate(spec, null, agg, functionFields); } public TridentState persistentAggregate(StateFactory stateFactory, Fields inputFields, ReducerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), inputFields, agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, Fields inputFields, ReducerAggregator agg, Fields functionFields) { projectionValidation(inputFields); return global().partitionPersist(spec, inputFields, new ReducerAggStateUpdater(agg), functionFields); } public Stream stateQuery(TridentState state, QueryFunction function, Fields functionFields) { return stateQuery(state, null, function, functionFields); } @Override public Stream toStream() { return this; } @Override public Fields getOutputFields() { return _node.allOutputFields; } static class BatchGlobalAggScheme implements GlobalAggregationScheme { @Override public IAggregatableStream aggPartition(Stream s) { return s.batchGlobal(); } @Override public BatchToPartition singleEmitPartitioner() { return new IndexHashBatchToPartition(); } } static class GlobalAggScheme implements GlobalAggregationScheme { @Override public IAggregatableStream aggPartition(Stream s) { return s.global(); } @Override public BatchToPartition singleEmitPartitioner() { return new GlobalBatchToPartition(); } } private void projectionValidation(Fields projFields) { if (projFields == null) { return; } Fields allFields = this.getOutputFields(); for (String field : projFields) { if (!allFields.contains(field)) { throw new IllegalArgumentException("Trying to select non-existent field: '" + field + "' from stream containing fields fields: <" + allFields + ">"); } } } } ================================================ FILE: storm-core/src/jvm/storm/trident/TridentState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident; import storm.trident.planner.Node; public class TridentState { TridentTopology _topology; Node _node; protected TridentState(TridentTopology topology, Node node) { _topology = topology; _node = node; } public Stream newValuesStream() { return new Stream(_topology, _node.name, _node); } public TridentState parallelismHint(int parallelism) { _node.parallelismHint = parallelism; return this; } } ================================================ FILE: storm-core/src/jvm/storm/trident/TridentTopology.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident; import backtype.storm.Config; import backtype.storm.ILocalDRPC; import backtype.storm.drpc.DRPCSpout; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.generated.StormTopology; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.topology.BoltDeclarer; import backtype.storm.topology.IRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; import org.jgrapht.DirectedGraph; import org.jgrapht.UndirectedGraph; import org.jgrapht.alg.ConnectivityInspector; import org.jgrapht.graph.DefaultDirectedGraph; import org.jgrapht.graph.Pseudograph; import storm.trident.drpc.ReturnResultsReducer; import storm.trident.fluent.GroupedStream; import storm.trident.fluent.IAggregatableStream; import storm.trident.fluent.UniqueIdGen; import storm.trident.graph.GraphGrouper; import storm.trident.graph.Group; import storm.trident.operation.GroupedMultiReducer; import storm.trident.operation.MultiReducer; import storm.trident.operation.impl.FilterExecutor; import storm.trident.operation.impl.GroupedMultiReducerExecutor; import storm.trident.operation.impl.IdentityMultiReducer; import storm.trident.operation.impl.JoinerMultiReducer; import storm.trident.operation.impl.TrueFilter; import storm.trident.partition.IdentityGrouping; import storm.trident.planner.Node; import storm.trident.planner.NodeStateInfo; import storm.trident.planner.PartitionNode; import storm.trident.planner.ProcessorNode; import storm.trident.planner.SpoutNode; import storm.trident.planner.SubtopologyBolt; import storm.trident.planner.processor.EachProcessor; import storm.trident.planner.processor.MultiReducerProcessor; import storm.trident.spout.BatchSpoutExecutor; import storm.trident.spout.IBatchSpout; import storm.trident.spout.IOpaquePartitionedTridentSpout; import storm.trident.spout.IPartitionedTridentSpout; import storm.trident.spout.ITridentSpout; import storm.trident.spout.OpaquePartitionedTridentSpoutExecutor; import storm.trident.spout.PartitionedTridentSpoutExecutor; import storm.trident.spout.RichSpoutBatchExecutor; import storm.trident.state.StateFactory; import storm.trident.state.StateSpec; import storm.trident.topology.TridentTopologyBuilder; import storm.trident.util.ErrorEdgeFactory; import storm.trident.util.IndexedEdge; import storm.trident.util.TridentUtils; // graph with 3 kinds of nodes: // operation, partition, or spout // all operations have finishBatch and can optionally be committers public class TridentTopology { //TODO: add a method for drpc stream, needs to know how to automatically do returnresults, etc // is it too expensive to do a batch per drpc request? DefaultDirectedGraph _graph; Map> _colocate = new HashMap(); UniqueIdGen _gen; public TridentTopology() { _graph = new DefaultDirectedGraph(new ErrorEdgeFactory()); _gen = new UniqueIdGen(); } private TridentTopology(DefaultDirectedGraph graph, Map> colocate, UniqueIdGen gen) { _graph = graph; _colocate = colocate; _gen = gen; } // automatically turn it into a batch spout, should take parameters as to how much to batch // public Stream newStream(IRichSpout spout) { // Node n = new SpoutNode(getUniqueStreamId(), TridentUtils.getSingleOutputStreamFields(spout), null, spout, SpoutNode.SpoutType.BATCH); // return addNode(n); // } public Stream newStream(String txId, IRichSpout spout) { return newStream(txId, new RichSpoutBatchExecutor(spout)); } public Stream newStream(String txId, IBatchSpout spout) { Node n = new SpoutNode(getUniqueStreamId(), spout.getOutputFields(), txId, spout, SpoutNode.SpoutType.BATCH); return addNode(n); } public Stream newStream(String txId, ITridentSpout spout) { Node n = new SpoutNode(getUniqueStreamId(), spout.getOutputFields(), txId, spout, SpoutNode.SpoutType.BATCH); return addNode(n); } public Stream newStream(String txId, IPartitionedTridentSpout spout) { return newStream(txId, new PartitionedTridentSpoutExecutor(spout)); } public Stream newStream(String txId, IOpaquePartitionedTridentSpout spout) { return newStream(txId, new OpaquePartitionedTridentSpoutExecutor(spout)); } public Stream newDRPCStream(String function) { return newDRPCStream(new DRPCSpout(function)); } public Stream newDRPCStream(String function, ILocalDRPC server) { DRPCSpout spout; if(server==null) { spout = new DRPCSpout(function); } else { spout = new DRPCSpout(function, server); } return newDRPCStream(spout); } private Stream newDRPCStream(DRPCSpout spout) { // TODO: consider adding a shuffle grouping after the spout to avoid so much routing of the args/return-info all over the place // (at least until its possible to just pack bolt logic into the spout itself) Node n = new SpoutNode(getUniqueStreamId(), TridentUtils.getSingleOutputStreamFields(spout), null, spout, SpoutNode.SpoutType.DRPC); Stream nextStream = addNode(n); // later on, this will be joined back with return-info and all the results return nextStream.project(new Fields("args")); } public TridentState newStaticState(StateFactory factory) { return newStaticState(new StateSpec(factory)); } public TridentState newStaticState(StateSpec spec) { String stateId = getUniqueStateId(); Node n = new Node(getUniqueStreamId(), null, new Fields()); n.stateInfo = new NodeStateInfo(stateId, spec); registerNode(n); return new TridentState(this, n); } public Stream multiReduce(Stream s1, Stream s2, MultiReducer function, Fields outputFields) { return multiReduce(Arrays.asList(s1, s2), function, outputFields); } public Stream multiReduce(Fields inputFields1, Stream s1, Fields inputFields2, Stream s2, MultiReducer function, Fields outputFields) { return multiReduce(Arrays.asList(inputFields1, inputFields2), Arrays.asList(s1, s2), function, outputFields); } public Stream multiReduce(GroupedStream s1, GroupedStream s2, GroupedMultiReducer function, Fields outputFields) { return multiReduce(Arrays.asList(s1, s2), function, outputFields); } public Stream multiReduce(Fields inputFields1, GroupedStream s1, Fields inputFields2, GroupedStream s2, GroupedMultiReducer function, Fields outputFields) { return multiReduce(Arrays.asList(inputFields1, inputFields2), Arrays.asList(s1, s2), function, outputFields); } public Stream multiReduce(List streams, MultiReducer function, Fields outputFields) { return multiReduce(getAllOutputFields(streams), streams, function, outputFields); } public Stream multiReduce(List streams, GroupedMultiReducer function, Fields outputFields) { return multiReduce(getAllOutputFields(streams), streams, function, outputFields); } public Stream multiReduce(List inputFields, List streams, MultiReducer function, Fields outputFields) { List names = new ArrayList(); for(Stream s: streams) { if(s._name!=null) { names.add(s._name); } } Node n = new ProcessorNode(getUniqueStreamId(), Utils.join(names, "-"), outputFields, outputFields, new MultiReducerProcessor(inputFields, function)); return addSourcedNode(streams, n); } public Stream multiReduce(List inputFields, List groupedStreams, GroupedMultiReducer function, Fields outputFields) { List fullInputFields = new ArrayList(); List streams = new ArrayList(); List fullGroupFields = new ArrayList(); for(int i=0; i streams) { return multiReduce(streams, new IdentityMultiReducer(), outputFields); } public Stream merge(Stream... streams) { return merge(Arrays.asList(streams)); } public Stream merge(List streams) { return merge(streams.get(0).getOutputFields(), streams); } public Stream join(Stream s1, Fields joinFields1, Stream s2, Fields joinFields2, Fields outFields) { return join(Arrays.asList(s1, s2), Arrays.asList(joinFields1, joinFields2), outFields); } public Stream join(List streams, List joinFields, Fields outFields) { return join(streams, joinFields, outFields, JoinType.INNER); } public Stream join(Stream s1, Fields joinFields1, Stream s2, Fields joinFields2, Fields outFields, JoinType type) { return join(Arrays.asList(s1, s2), Arrays.asList(joinFields1, joinFields2), outFields, type); } public Stream join(List streams, List joinFields, Fields outFields, JoinType type) { return join(streams, joinFields, outFields, repeat(streams.size(), type)); } public Stream join(Stream s1, Fields joinFields1, Stream s2, Fields joinFields2, Fields outFields, List mixed) { return join(Arrays.asList(s1, s2), Arrays.asList(joinFields1, joinFields2), outFields, mixed); } public Stream join(List streams, List joinFields, Fields outFields, List mixed) { return multiReduce(strippedInputFields(streams, joinFields), groupedStreams(streams, joinFields), new JoinerMultiReducer(mixed, joinFields.get(0).size(), strippedInputFields(streams, joinFields)), outFields); } public StormTopology build() { DefaultDirectedGraph graph = (DefaultDirectedGraph) _graph.clone(); completeDRPC(graph, _colocate, _gen); List spoutNodes = new ArrayList(); // can be regular nodes (static state) or processor nodes Set boltNodes = new HashSet(); for(Node n: graph.vertexSet()) { if(n instanceof SpoutNode) { spoutNodes.add((SpoutNode) n); } else if(!(n instanceof PartitionNode)) { boltNodes.add(n); } } Set initialGroups = new HashSet(); for(List colocate: _colocate.values()) { Group g = new Group(graph, colocate); boltNodes.removeAll(colocate); initialGroups.add(g); } for(Node n: boltNodes) { initialGroups.add(new Group(graph, n)); } GraphGrouper grouper = new GraphGrouper(graph, initialGroups); grouper.mergeFully(); Collection mergedGroups = grouper.getAllGroups(); // add identity partitions between groups for(IndexedEdge e: new HashSet(graph.edgeSet())) { if(!(e.source instanceof PartitionNode) && !(e.target instanceof PartitionNode)) { Group g1 = grouper.nodeGroup(e.source); Group g2 = grouper.nodeGroup(e.target); // g1 being null means the source is a spout node if(g1==null && !(e.source instanceof SpoutNode)) throw new RuntimeException("Planner exception: Null source group must indicate a spout node at this phase of planning"); if(g1==null || !g1.equals(g2)) { graph.removeEdge(e); PartitionNode pNode = makeIdentityPartition(e.source); graph.addVertex(pNode); graph.addEdge(e.source, pNode, new IndexedEdge(e.source, pNode, 0)); graph.addEdge(pNode, e.target, new IndexedEdge(pNode, e.target, e.index)); } } } // if one group subscribes to the same stream with same partitioning multiple times, // merge those together (otherwise can end up with many output streams created for that partitioning // if need to split into multiple output streams because of same input having different // partitioning to the group) // this is because can't currently merge splitting logic into a spout // not the most kosher algorithm here, since the grouper indexes are being trounced via the adding of nodes to random groups, but it // works out List forNewGroups = new ArrayList(); for(Group g: mergedGroups) { for(PartitionNode n: extraPartitionInputs(g)) { Node idNode = makeIdentityNode(n.allOutputFields); Node newPartitionNode = new PartitionNode(idNode.streamId, n.name, idNode.allOutputFields, n.thriftGrouping); Node parentNode = TridentUtils.getParent(graph, n); Set outgoing = graph.outgoingEdgesOf(n); graph.removeVertex(n); graph.addVertex(idNode); graph.addVertex(newPartitionNode); addEdge(graph, parentNode, idNode, 0); addEdge(graph, idNode, newPartitionNode, 0); for(IndexedEdge e: outgoing) { addEdge(graph, newPartitionNode, e.target, e.index); } Group parentGroup = grouper.nodeGroup(parentNode); if(parentGroup==null) { forNewGroups.add(idNode); } else { parentGroup.nodes.add(idNode); } } } // TODO: in the future, want a way to include this logic in the spout itself, // or make it unecessary by having storm include metadata about which grouping a tuple // came from for(Node n: forNewGroups) { grouper.addGroup(new Group(graph, n)); } // add in spouts as groups so we can get parallelisms for(Node n: spoutNodes) { grouper.addGroup(new Group(graph, n)); } grouper.reindex(); mergedGroups = grouper.getAllGroups(); Map batchGroupMap = new HashMap(); List> connectedComponents = new ConnectivityInspector(graph).connectedSets(); for(int i=0; i parallelisms = getGroupParallelisms(graph, grouper, mergedGroups); TridentTopologyBuilder builder = new TridentTopologyBuilder(); Map spoutIds = genSpoutIds(spoutNodes); Map boltIds = genBoltIds(mergedGroups); for(SpoutNode sn: spoutNodes) { Integer parallelism = parallelisms.get(grouper.nodeGroup(sn)); if(sn.type == SpoutNode.SpoutType.DRPC) { builder.setBatchPerTupleSpout(spoutIds.get(sn), sn.streamId, (IRichSpout) sn.spout, parallelism, batchGroupMap.get(sn)); } else { ITridentSpout s; if(sn.spout instanceof IBatchSpout) { s = new BatchSpoutExecutor((IBatchSpout)sn.spout); } else if(sn.spout instanceof ITridentSpout) { s = (ITridentSpout) sn.spout; } else { throw new RuntimeException("Regular rich spouts not supported yet... try wrapping in a RichSpoutBatchExecutor"); // TODO: handle regular rich spout without batches (need lots of updates to support this throughout) } builder.setSpout(spoutIds.get(sn), sn.streamId, sn.txId, s, parallelism, batchGroupMap.get(sn)); } } for(Group g: mergedGroups) { if(!isSpoutGroup(g)) { Integer p = parallelisms.get(g); Map streamToGroup = getOutputStreamBatchGroups(g, batchGroupMap); BoltDeclarer d = builder.setBolt(boltIds.get(g), new SubtopologyBolt(graph, g.nodes, batchGroupMap), p, committerBatches(g, batchGroupMap), streamToGroup); Collection inputs = uniquedSubscriptions(externalGroupInputs(g)); for(PartitionNode n: inputs) { Node parent = TridentUtils.getParent(graph, n); String componentId; if(parent instanceof SpoutNode) { componentId = spoutIds.get(parent); } else { componentId = boltIds.get(grouper.nodeGroup(parent)); } d.grouping(new GlobalStreamId(componentId, n.streamId), n.thriftGrouping); } } } return builder.buildTopology(); } private static void completeDRPC(DefaultDirectedGraph graph, Map> colocate, UniqueIdGen gen) { List> connectedComponents = new ConnectivityInspector(graph).connectedSets(); for(Set g: connectedComponents) { checkValidJoins(g); } TridentTopology helper = new TridentTopology(graph, colocate, gen); for(Set g: connectedComponents) { SpoutNode drpcNode = getDRPCSpoutNode(g); if(drpcNode!=null) { Stream lastStream = new Stream(helper, null, getLastAddedNode(g)); Stream s = new Stream(helper, null, drpcNode); helper.multiReduce( s.project(new Fields("return-info")) .batchGlobal(), lastStream.batchGlobal(), new ReturnResultsReducer(), new Fields()); } } } private static Node getLastAddedNode(Collection g) { Node ret = null; for(Node n: g) { if(ret==null || n.creationIndex > ret.creationIndex) { ret = n; } } return ret; } //returns null if it's not a drpc group private static SpoutNode getDRPCSpoutNode(Collection g) { for(Node n: g) { if(n instanceof SpoutNode) { SpoutNode.SpoutType type = ((SpoutNode) n).type; if(type==SpoutNode.SpoutType.DRPC) { return (SpoutNode) n; } } } return null; } private static void checkValidJoins(Collection g) { boolean hasDRPCSpout = false; boolean hasBatchSpout = false; for(Node n: g) { if(n instanceof SpoutNode) { SpoutNode.SpoutType type = ((SpoutNode) n).type; if(type==SpoutNode.SpoutType.BATCH) { hasBatchSpout = true; } else if(type==SpoutNode.SpoutType.DRPC) { hasDRPCSpout = true; } } } if(hasBatchSpout && hasDRPCSpout) { throw new RuntimeException("Cannot join DRPC stream with streams originating from other spouts"); } } private static boolean isSpoutGroup(Group g) { return g.nodes.size() == 1 && g.nodes.iterator().next() instanceof SpoutNode; } private static Collection uniquedSubscriptions(Set subscriptions) { Map ret = new HashMap(); for(PartitionNode n: subscriptions) { PartitionNode curr = ret.get(n.streamId); if(curr!=null && !curr.thriftGrouping.equals(n.thriftGrouping)) { throw new RuntimeException("Multiple subscriptions to the same stream with different groupings. Should be impossible since that is explicitly guarded against."); } ret.put(n.streamId, n); } return ret.values(); } private static Map genSpoutIds(Collection spoutNodes) { Map ret = new HashMap(); int ctr = 0; for(SpoutNode n: spoutNodes) { ret.put(n, "spout" + ctr); ctr++; } return ret; } private static Map genBoltIds(Collection groups) { Map ret = new HashMap(); int ctr = 0; for(Group g: groups) { if(!isSpoutGroup(g)) { List name = new ArrayList(); name.add("b"); name.add("" + ctr); String groupName = getGroupName(g); if(groupName!=null && !groupName.isEmpty()) { name.add(getGroupName(g)); } ret.put(g, Utils.join(name, "-")); ctr++; } } return ret; } private static String getGroupName(Group g) { TreeMap sortedNames = new TreeMap(); for(Node n: g.nodes) { if(n.name!=null) { sortedNames.put(n.creationIndex, n.name); } } List names = new ArrayList(); String prevName = null; for(String n: sortedNames.values()) { if(prevName==null || !n.equals(prevName)) { prevName = n; names.add(n); } } return Utils.join(names, "-"); } private static Map getOutputStreamBatchGroups(Group g, Map batchGroupMap) { Map ret = new HashMap(); Set externalGroupOutputs = externalGroupOutputs(g); for(PartitionNode n: externalGroupOutputs) { ret.put(n.streamId, batchGroupMap.get(n)); } return ret; } private static Set committerBatches(Group g, Map batchGroupMap) { Set ret = new HashSet(); for(Node n: g.nodes) { if(n instanceof ProcessorNode) { if(((ProcessorNode) n).committer) { ret.add(batchGroupMap.get(n)); } } } return ret; } private static Map getGroupParallelisms(DirectedGraph graph, GraphGrouper grouper, Collection groups) { UndirectedGraph equivs = new Pseudograph(Object.class); for(Group g: groups) { equivs.addVertex(g); } for(Group g: groups) { for(PartitionNode n: externalGroupInputs(g)) { if(isIdentityPartition(n)) { Node parent = TridentUtils.getParent(graph, n); Group parentGroup = grouper.nodeGroup(parent); if(parentGroup!=null && !parentGroup.equals(g)) { equivs.addEdge(parentGroup, g); } } } } Map ret = new HashMap(); List> equivGroups = new ConnectivityInspector(equivs).connectedSets(); for(Set equivGroup: equivGroups) { Integer fixedP = getFixedParallelism(equivGroup); Integer maxP = getMaxParallelism(equivGroup); if(fixedP!=null && maxP!=null && maxP < fixedP) { throw new RuntimeException("Parallelism is fixed to " + fixedP + " but max parallelism is less than that: " + maxP); } Integer p = 1; for(Group g: equivGroup) { for(Node n: g.nodes) { if(n.parallelismHint!=null) { p = Math.max(p, n.parallelismHint); } } } if(maxP!=null) p = Math.min(maxP, p); if(fixedP!=null) p = fixedP; for(Group g: equivGroup) { ret.put(g, p); } } return ret; } private static Integer getMaxParallelism(Set groups) { Integer ret = null; for(Group g: groups) { if(isSpoutGroup(g)) { SpoutNode n = (SpoutNode) g.nodes.iterator().next(); Map conf = getSpoutComponentConfig(n.spout); if(conf==null) conf = new HashMap(); Number maxP = (Number) conf.get(Config.TOPOLOGY_MAX_TASK_PARALLELISM); if(maxP!=null) { if(ret==null) ret = maxP.intValue(); else ret = Math.min(ret, maxP.intValue()); } } } return ret; } private static Map getSpoutComponentConfig(Object spout) { if(spout instanceof IRichSpout) { return ((IRichSpout) spout).getComponentConfiguration(); } else if (spout instanceof IBatchSpout) { return ((IBatchSpout) spout).getComponentConfiguration(); } else { return ((ITridentSpout) spout).getComponentConfiguration(); } } private static Integer getFixedParallelism(Set groups) { Integer ret = null; for(Group g: groups) { for(Node n: g.nodes) { if(n.stateInfo != null && n.stateInfo.spec.requiredNumPartitions!=null) { int reqPartitions = n.stateInfo.spec.requiredNumPartitions; if(ret!=null && ret!=reqPartitions) { throw new RuntimeException("Cannot have one group have fixed parallelism of two different values"); } ret = reqPartitions; } } } return ret; } private static boolean isIdentityPartition(PartitionNode n) { Grouping g = n.thriftGrouping; if(g.is_set_custom_serialized()) { CustomStreamGrouping csg = (CustomStreamGrouping) Utils.deserialize(g.get_custom_serialized()); return csg instanceof IdentityGrouping; } return false; } private static void addEdge(DirectedGraph g, Object source, Object target, int index) { g.addEdge(source, target, new IndexedEdge(source, target, index)); } private Node makeIdentityNode(Fields allOutputFields) { return new ProcessorNode(getUniqueStreamId(), null, allOutputFields, new Fields(), new EachProcessor(new Fields(), new FilterExecutor(new TrueFilter()))); } private static List extraPartitionInputs(Group g) { List ret = new ArrayList(); Set inputs = externalGroupInputs(g); Map> grouped = new HashMap(); for(PartitionNode n: inputs) { if(!grouped.containsKey(n.streamId)) { grouped.put(n.streamId, new ArrayList()); } grouped.get(n.streamId).add(n); } for(List group: grouped.values()) { PartitionNode anchor = group.get(0); for(int i=1; i externalGroupInputs(Group g) { Set ret = new HashSet(); for(Node n: g.incomingNodes()) { if(n instanceof PartitionNode) { ret.add((PartitionNode) n); } } return ret; } private static Set externalGroupOutputs(Group g) { Set ret = new HashSet(); for(Node n: g.outgoingNodes()) { if(n instanceof PartitionNode) { ret.add((PartitionNode) n); } } return ret; } private static PartitionNode makeIdentityPartition(Node basis) { return new PartitionNode(basis.streamId, basis.name, basis.allOutputFields, Grouping.custom_serialized(Utils.serialize(new IdentityGrouping()))); } protected String getUniqueStreamId() { return _gen.getUniqueStreamId(); } protected String getUniqueStateId() { return _gen.getUniqueStateId(); } protected void registerNode(Node n) { _graph.addVertex(n); if(n.stateInfo!=null) { String id = n.stateInfo.id; if(!_colocate.containsKey(id)) { _colocate.put(id, new ArrayList()); } _colocate.get(id).add(n); } } protected Stream addNode(Node n) { registerNode(n); return new Stream(this, n.name, n); } protected void registerSourcedNode(List sources, Node newNode) { registerNode(newNode); int streamIndex = 0; for(Stream s: sources) { _graph.addEdge(s._node, newNode, new IndexedEdge(s._node, newNode, streamIndex)); streamIndex++; } } protected Stream addSourcedNode(List sources, Node newNode) { registerSourcedNode(sources, newNode); return new Stream(this, newNode.name, newNode); } protected TridentState addSourcedStateNode(List sources, Node newNode) { registerSourcedNode(sources, newNode); return new TridentState(this, newNode); } protected Stream addSourcedNode(Stream source, Node newNode) { return addSourcedNode(Arrays.asList(source), newNode); } protected TridentState addSourcedStateNode(Stream source, Node newNode) { return addSourcedStateNode(Arrays.asList(source), newNode); } private static List getAllOutputFields(List streams) { List ret = new ArrayList(); for(Object o: streams) { ret.add(((IAggregatableStream) o).getOutputFields()); } return ret; } private static List groupedStreams(List streams, List joinFields) { List ret = new ArrayList(); for(int i=0; i strippedInputFields(List streams, List joinFields) { List ret = new ArrayList(); for(int i=0; i repeat(int n, JoinType type) { List ret = new ArrayList(); for(int i=0; i { public static class ReturnResultsState { List results = new ArrayList(); String returnInfo; @Override public String toString() { return ToStringBuilder.reflectionToString(this); } } boolean local; Map _clients = new HashMap(); @Override public void prepare(Map conf, TridentMultiReducerContext context) { local = conf.get(Config.STORM_CLUSTER_MODE).equals("local"); } @Override public ReturnResultsState init(TridentCollector collector) { return new ReturnResultsState(); } @Override public void execute(ReturnResultsState state, int streamIndex, TridentTuple input, TridentCollector collector) { if(streamIndex==0) { state.returnInfo = input.getString(0); } else { state.results.add(input); } } @Override public void complete(ReturnResultsState state, TridentCollector collector) { // only one of the multireducers will receive the tuples if(state.returnInfo!=null) { String result = JSONValue.toJSONString(state.results); Map retMap = (Map) JSONValue.parse(state.returnInfo); final String host = (String) retMap.get("host"); final int port = Utils.getInt(retMap.get("port")); String id = (String) retMap.get("id"); DistributedRPCInvocations.Iface client; if(local) { client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host); } else { List server = new ArrayList() {{ add(host); add(port); }}; if(!_clients.containsKey(server)) { _clients.put(server, new DRPCInvocationsClient(host, port)); } client = _clients.get(server); } try { client.result(id, result); } catch(TException e) { collector.reportError(e); } } } @Override public void cleanup() { for(DRPCInvocationsClient c: _clients.values()) { c.close(); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/ChainedAggregatorDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Set; import storm.trident.Stream; import storm.trident.operation.Aggregator; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.ReducerAggregator; import storm.trident.operation.impl.ChainedAggregatorImpl; import storm.trident.operation.impl.CombinerAggregatorCombineImpl; import storm.trident.operation.impl.CombinerAggregatorInitImpl; import storm.trident.operation.impl.ReducerAggregatorImpl; import storm.trident.operation.impl.SingleEmitAggregator; import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; import storm.trident.tuple.ComboList; public class ChainedAggregatorDeclarer implements ChainedFullAggregatorDeclarer, ChainedPartitionAggregatorDeclarer { public static interface AggregationPartition { Stream partition(Stream input); } private static enum AggType { PARTITION, FULL, FULL_COMBINE } // inputFields can be equal to outFields, but multiple aggregators cannot have intersection outFields private static class AggSpec { Fields inFields; Aggregator agg; Fields outFields; public AggSpec(Fields inFields, Aggregator agg, Fields outFields) { this.inFields = inFields; this.agg = agg; this.outFields = outFields; } } List _aggs = new ArrayList(); IAggregatableStream _stream; AggType _type = null; GlobalAggregationScheme _globalScheme; public ChainedAggregatorDeclarer(IAggregatableStream stream, GlobalAggregationScheme globalScheme) { _stream = stream; _globalScheme = globalScheme; } public Stream chainEnd() { Fields[] inputFields = new Fields[_aggs.size()]; Aggregator[] aggs = new Aggregator[_aggs.size()]; int[] outSizes = new int[_aggs.size()]; List allOutFields = new ArrayList(); Set allInFields = new HashSet(); for(int i=0; i<_aggs.size(); i++) { AggSpec spec = _aggs.get(i); Fields infields = spec.inFields; if(infields==null) infields = new Fields(); Fields outfields = spec.outFields; if(outfields==null) outfields = new Fields(); inputFields[i] = infields; aggs[i] = spec.agg; outSizes[i] = outfields.size(); allOutFields.addAll(outfields.toList()); allInFields.addAll(infields.toList()); } if(new HashSet(allOutFields).size() != allOutFields.size()) { throw new IllegalArgumentException("Output fields for chained aggregators must be distinct: " + allOutFields.toString()); } Fields inFields = new Fields(new ArrayList(allInFields)); Fields outFields = new Fields(allOutFields); Aggregator combined = new ChainedAggregatorImpl(aggs, inputFields, new ComboList.Factory(outSizes)); if(_type!=AggType.FULL) { _stream = _stream.partitionAggregate(inFields, combined, outFields); } if(_type!=AggType.PARTITION) { _stream = _globalScheme.aggPartition(_stream); BatchToPartition singleEmit = _globalScheme.singleEmitPartitioner(); Aggregator toAgg = combined; if(singleEmit!=null) { toAgg = new SingleEmitAggregator(combined, singleEmit); } // this assumes that inFields and outFields are the same for combineragg // assumption also made above _stream = _stream.partitionAggregate(inFields, toAgg, outFields); } return _stream.toStream(); } public ChainedPartitionAggregatorDeclarer partitionAggregate(Aggregator agg, Fields functionFields) { return partitionAggregate(null, agg, functionFields); } public ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields) { _type = AggType.PARTITION; _aggs.add(new AggSpec(inputFields, agg, functionFields)); return this; } public ChainedPartitionAggregatorDeclarer partitionAggregate(CombinerAggregator agg, Fields functionFields) { return partitionAggregate(null, agg, functionFields); } public ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields) { initCombiner(inputFields, agg, functionFields); return partitionAggregate(functionFields, new CombinerAggregatorCombineImpl(agg), functionFields); } public ChainedPartitionAggregatorDeclarer partitionAggregate(ReducerAggregator agg, Fields functionFields) { return partitionAggregate(null, agg, functionFields); } public ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields) { return partitionAggregate(inputFields, new ReducerAggregatorImpl(agg), functionFields); } public ChainedFullAggregatorDeclarer aggregate(Aggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public ChainedFullAggregatorDeclarer aggregate(Fields inputFields, Aggregator agg, Fields functionFields) { return aggregate(inputFields, agg, functionFields, false); } private ChainedFullAggregatorDeclarer aggregate(Fields inputFields, Aggregator agg, Fields functionFields, boolean isCombiner) { if(isCombiner) { if(_type == null) { _type = AggType.FULL_COMBINE; } } else { _type = AggType.FULL; } _aggs.add(new AggSpec(inputFields, agg, functionFields)); return this; } public ChainedFullAggregatorDeclarer aggregate(CombinerAggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public ChainedFullAggregatorDeclarer aggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields) { initCombiner(inputFields, agg, functionFields); return aggregate(functionFields, new CombinerAggregatorCombineImpl(agg), functionFields, true); } public ChainedFullAggregatorDeclarer aggregate(ReducerAggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public ChainedFullAggregatorDeclarer aggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields) { return aggregate(inputFields, new ReducerAggregatorImpl(agg), functionFields); } private void initCombiner(Fields inputFields, CombinerAggregator agg, Fields functionFields) { _stream = _stream.each(inputFields, new CombinerAggregatorInitImpl(agg), functionFields); } } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/ChainedFullAggregatorDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; import backtype.storm.tuple.Fields; import storm.trident.operation.Aggregator; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.ReducerAggregator; public interface ChainedFullAggregatorDeclarer extends IChainedAggregatorDeclarer { ChainedFullAggregatorDeclarer aggregate(Aggregator agg, Fields functionFields); ChainedFullAggregatorDeclarer aggregate(Fields inputFields, Aggregator agg, Fields functionFields); ChainedFullAggregatorDeclarer aggregate(CombinerAggregator agg, Fields functionFields); ChainedFullAggregatorDeclarer aggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields); ChainedFullAggregatorDeclarer aggregate(ReducerAggregator agg, Fields functionFields); ChainedFullAggregatorDeclarer aggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields); } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/ChainedPartitionAggregatorDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; import backtype.storm.tuple.Fields; import storm.trident.operation.Aggregator; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.ReducerAggregator; public interface ChainedPartitionAggregatorDeclarer extends IChainedAggregatorDeclarer { ChainedPartitionAggregatorDeclarer partitionAggregate(Aggregator agg, Fields functionFields); ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields); ChainedPartitionAggregatorDeclarer partitionAggregate(CombinerAggregator agg, Fields functionFields); ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields); ChainedPartitionAggregatorDeclarer partitionAggregate(ReducerAggregator agg, Fields functionFields); ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields); } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/GlobalAggregationScheme.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; public interface GlobalAggregationScheme { IAggregatableStream aggPartition(S stream); // how to partition for second stage of aggregation BatchToPartition singleEmitPartitioner(); // return null if it's not single emit } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/GroupedStream.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; import backtype.storm.tuple.Fields; import storm.trident.Stream; import storm.trident.TridentState; import storm.trident.operation.Aggregator; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.Function; import storm.trident.operation.ReducerAggregator; import storm.trident.operation.impl.GroupedAggregator; import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; import storm.trident.state.QueryFunction; import storm.trident.state.StateFactory; import storm.trident.state.StateSpec; import storm.trident.state.map.MapCombinerAggStateUpdater; import storm.trident.state.map.MapReducerAggStateUpdater; import storm.trident.util.TridentUtils; public class GroupedStream implements IAggregatableStream, GlobalAggregationScheme { Fields _groupFields; Stream _stream; public GroupedStream(Stream stream, Fields groupFields) { _groupFields = groupFields; _stream = stream; } public GroupedStream name(String name) { return new GroupedStream(_stream.name(name), _groupFields); } public ChainedAggregatorDeclarer chainedAgg() { return new ChainedAggregatorDeclarer(this, this); } public Stream aggregate(Aggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public Stream aggregate(Fields inputFields, Aggregator agg, Fields functionFields) { return new ChainedAggregatorDeclarer(this, this) .aggregate(inputFields, agg, functionFields) .chainEnd(); } public Stream aggregate(CombinerAggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public Stream aggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields) { return new ChainedAggregatorDeclarer(this, this) .aggregate(inputFields, agg, functionFields) .chainEnd(); } public Stream aggregate(ReducerAggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } public Stream aggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields) { return new ChainedAggregatorDeclarer(this, this) .aggregate(inputFields, agg, functionFields) .chainEnd(); } public TridentState persistentAggregate(StateFactory stateFactory, CombinerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, CombinerAggregator agg, Fields functionFields) { return persistentAggregate(spec, null, agg, functionFields); } public TridentState persistentAggregate(StateFactory stateFactory, Fields inputFields, CombinerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), inputFields, agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, Fields inputFields, CombinerAggregator agg, Fields functionFields) { return aggregate(inputFields, agg, functionFields) .partitionPersist(spec, TridentUtils.fieldsUnion(_groupFields, functionFields), new MapCombinerAggStateUpdater(agg, _groupFields, functionFields), TridentUtils.fieldsConcat(_groupFields, functionFields)); } public TridentState persistentAggregate(StateFactory stateFactory, Fields inputFields, ReducerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), inputFields, agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, Fields inputFields, ReducerAggregator agg, Fields functionFields) { return _stream.partitionBy(_groupFields) .partitionPersist(spec, TridentUtils.fieldsUnion(_groupFields, inputFields), new MapReducerAggStateUpdater(agg, _groupFields, inputFields), TridentUtils.fieldsConcat(_groupFields, functionFields)); } public Stream stateQuery(TridentState state, Fields inputFields, QueryFunction function, Fields functionFields) { return _stream.partitionBy(_groupFields) .stateQuery(state, inputFields, function, functionFields); } public TridentState persistentAggregate(StateFactory stateFactory, ReducerAggregator agg, Fields functionFields) { return persistentAggregate(new StateSpec(stateFactory), agg, functionFields); } public TridentState persistentAggregate(StateSpec spec, ReducerAggregator agg, Fields functionFields) { return persistentAggregate(spec, null, agg, functionFields); } public Stream stateQuery(TridentState state, QueryFunction function, Fields functionFields) { return stateQuery(state, null, function, functionFields); } @Override public IAggregatableStream each(Fields inputFields, Function function, Fields functionFields) { Stream s = _stream.each(inputFields, function, functionFields); return new GroupedStream(s, _groupFields); } @Override public IAggregatableStream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields) { Aggregator groupedAgg = new GroupedAggregator(agg, _groupFields, inputFields, functionFields.size()); Fields allInFields = TridentUtils.fieldsUnion(_groupFields, inputFields); Fields allOutFields = TridentUtils.fieldsConcat(_groupFields, functionFields); Stream s = _stream.partitionAggregate(allInFields, groupedAgg, allOutFields); return new GroupedStream(s, _groupFields); } @Override public IAggregatableStream aggPartition(GroupedStream s) { return new GroupedStream(s._stream.partitionBy(_groupFields), _groupFields); } @Override public Stream toStream() { return _stream; } @Override public Fields getOutputFields() { return _stream.getOutputFields(); } public Fields getGroupFields() { return _groupFields; } @Override public BatchToPartition singleEmitPartitioner() { return null; } } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/IAggregatableStream.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; import backtype.storm.tuple.Fields; import storm.trident.Stream; import storm.trident.operation.Aggregator; import storm.trident.operation.Function; import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; public interface IAggregatableStream { IAggregatableStream each(Fields inputFields, Function function, Fields functionFields); IAggregatableStream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields); Stream toStream(); Fields getOutputFields(); } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/IChainedAggregatorDeclarer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; import storm.trident.Stream; public interface IChainedAggregatorDeclarer { Stream chainEnd(); } ================================================ FILE: storm-core/src/jvm/storm/trident/fluent/UniqueIdGen.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.fluent; public class UniqueIdGen { int _streamCounter = 0; public String getUniqueStreamId() { _streamCounter++; return "s" + _streamCounter; } int _stateCounter = 0; public String getUniqueStateId() { _stateCounter++; return "state" + _stateCounter; } } ================================================ FILE: storm-core/src/jvm/storm/trident/graph/GraphGrouper.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.graph; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; import org.jgrapht.DirectedGraph; import storm.trident.planner.Node; import storm.trident.util.IndexedEdge; public class GraphGrouper { DirectedGraph graph; Set currGroups; Map groupIndex = new HashMap(); public GraphGrouper(DirectedGraph graph, Collection initialGroups) { this.graph = graph; this.currGroups = new HashSet(initialGroups); reindex(); } public Collection getAllGroups() { return currGroups; } public void addGroup(Group g) { currGroups.add(g); } public void reindex() { groupIndex.clear(); for(Group g: currGroups) { for(Node n: g.nodes) { groupIndex.put(n, g); } } } public void mergeFully() { boolean somethingHappened = true; while(somethingHappened) { somethingHappened = false; for(Group g: currGroups) { Collection outgoingGroups = outgoingGroups(g); if(outgoingGroups.size()==1) { Group out = outgoingGroups.iterator().next(); if(out!=null) { merge(g, out); somethingHappened = true; break; } } Collection incomingGroups = incomingGroups(g); if(incomingGroups.size()==1) { Group in = incomingGroups.iterator().next(); if(in!=null) { merge(g, in); somethingHappened = true; break; } } } } } private void merge(Group g1, Group g2) { Group newGroup = new Group(g1, g2); currGroups.remove(g1); currGroups.remove(g2); currGroups.add(newGroup); for(Node n: newGroup.nodes) { groupIndex.put(n, newGroup); } } public Collection outgoingGroups(Group g) { Set ret = new HashSet(); for(Node n: g.outgoingNodes()) { Group other = nodeGroup(n); if(other==null || !other.equals(g)) { ret.add(other); } } return ret; } public Collection incomingGroups(Group g) { Set ret = new HashSet(); for(Node n: g.incomingNodes()) { Group other = nodeGroup(n); if(other==null || !other.equals(g)) { ret.add(other); } } return ret; } public Group nodeGroup(Node n) { return groupIndex.get(n); } } ================================================ FILE: storm-core/src/jvm/storm/trident/graph/Group.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.graph; import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.UUID; import org.jgrapht.DirectedGraph; import storm.trident.planner.Node; import storm.trident.util.IndexedEdge; import storm.trident.util.TridentUtils; public class Group { public Set nodes = new HashSet(); private DirectedGraph graph; private String id; public Group(DirectedGraph graph, List nodes) { init(graph); this.nodes.addAll(nodes); this.graph = graph; } public Group(DirectedGraph graph, Node n) { this(graph, Arrays.asList(n)); } public Group(Group g1, Group g2) { init(g1.graph); nodes.addAll(g1.nodes); nodes.addAll(g2.nodes); } private void init(DirectedGraph graph) { this.graph = graph; this.id = UUID.randomUUID().toString(); } public Set outgoingNodes() { Set ret = new HashSet(); for(Node n: nodes) { ret.addAll(TridentUtils.getChildren(graph, n)); } return ret; } public Set incomingNodes() { Set ret = new HashSet(); for(Node n: nodes) { ret.addAll(TridentUtils.getParents(graph, n)); } return ret; } @Override public int hashCode() { return id.hashCode(); } @Override public boolean equals(Object o) { return id.equals(((Group) o).id); } @Override public String toString() { return nodes.toString(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/Aggregator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import storm.trident.tuple.TridentTuple; public interface Aggregator extends Operation { T init(Object batchId, TridentCollector collector); void aggregate(T val, TridentTuple tuple, TridentCollector collector); void complete(T val, TridentCollector collector); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/Assembly.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import storm.trident.Stream; public interface Assembly { Stream apply(Stream input); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/BaseAggregator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; public abstract class BaseAggregator extends BaseOperation implements Aggregator { } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/BaseFilter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; public abstract class BaseFilter extends BaseOperation implements Filter { } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/BaseFunction.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; public abstract class BaseFunction extends BaseOperation implements Function { } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/BaseMultiReducer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.util.Map; public abstract class BaseMultiReducer implements MultiReducer { @Override public void prepare(Map conf, TridentMultiReducerContext context) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/BaseOperation.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.util.Map; public class BaseOperation implements Operation { @Override public void prepare(Map conf, TridentOperationContext context) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/CombinerAggregator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.io.Serializable; import storm.trident.tuple.TridentTuple; // doesn't manipulate tuples (lists of stuff) so that things like aggregating into // cassandra is cleaner (don't need lists everywhere, just store the single value there) public interface CombinerAggregator extends Serializable { T init(TridentTuple tuple); T combine(T val1, T val2); T zero(); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/EachOperation.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; public interface EachOperation extends Operation { } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/Filter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import storm.trident.tuple.TridentTuple; public interface Filter extends EachOperation { boolean isKeep(TridentTuple tuple); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/Function.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import storm.trident.tuple.TridentTuple; public interface Function extends EachOperation { void execute(TridentTuple tuple, TridentCollector collector); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/GroupedMultiReducer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.io.Serializable; import java.util.Map; import storm.trident.tuple.TridentTuple; public interface GroupedMultiReducer extends Serializable { void prepare(Map conf, TridentMultiReducerContext context); T init(TridentCollector collector, TridentTuple group); void execute(T state, int streamIndex, TridentTuple group, TridentTuple input, TridentCollector collector); void complete(T state, TridentTuple group, TridentCollector collector); void cleanup(); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/MultiReducer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.io.Serializable; import java.util.Map; import storm.trident.tuple.TridentTuple; public interface MultiReducer extends Serializable { void prepare(Map conf, TridentMultiReducerContext context); T init(TridentCollector collector); void execute(T state, int streamIndex, TridentTuple input, TridentCollector collector); void complete(T state, TridentCollector collector); void cleanup(); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/Operation.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.io.Serializable; import java.util.Map; public interface Operation extends Serializable { void prepare(Map conf, TridentOperationContext context); void cleanup(); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/ReducerAggregator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.io.Serializable; import storm.trident.tuple.TridentTuple; public interface ReducerAggregator extends Serializable { T init(); T reduce(T curr, TridentTuple tuple); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/TridentCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import java.util.List; public interface TridentCollector { void emit(List values); void reportError(Throwable t); } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/TridentMultiReducerContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import backtype.storm.tuple.Fields; import java.util.List; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class TridentMultiReducerContext { List _factories; public TridentMultiReducerContext(List factories) { _factories = factories; } public ProjectionFactory makeProjectionFactory(int streamIndex, Fields fields) { return new ProjectionFactory(_factories.get(streamIndex), fields); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/TridentOperationContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation; import backtype.storm.metric.api.CombinedMetric; import backtype.storm.metric.api.ICombiner; import backtype.storm.metric.api.IMetric; import backtype.storm.metric.api.IReducer; import backtype.storm.metric.api.ReducedMetric; import backtype.storm.task.IMetricsContext; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class TridentOperationContext implements IMetricsContext{ TridentTuple.Factory _factory; TopologyContext _topoContext; public TridentOperationContext(TopologyContext topoContext, TridentTuple.Factory factory) { _factory = factory; _topoContext = topoContext; } public TridentOperationContext(TridentOperationContext parent, TridentTuple.Factory factory) { this(parent._topoContext, factory); } public ProjectionFactory makeProjectionFactory(Fields fields) { return new ProjectionFactory(_factory, fields); } public int numPartitions() { return _topoContext.getComponentTasks(_topoContext.getThisComponentId()).size(); } public int getPartitionIndex() { return _topoContext.getThisTaskIndex(); } public T registerMetric(String name, T metric, int timeBucketSizeInSecs) { return _topoContext.registerMetric(name, metric, timeBucketSizeInSecs); } public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) { return _topoContext.registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs); } public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) { return _topoContext.registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/builtin/Count.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.builtin; import storm.trident.operation.CombinerAggregator; import storm.trident.tuple.TridentTuple; public class Count implements CombinerAggregator { @Override public Long init(TridentTuple tuple) { return 1L; } @Override public Long combine(Long val1, Long val2) { return val1 + val2; } @Override public Long zero() { return 0L; } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/builtin/Debug.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.builtin; import storm.trident.operation.BaseFilter; import storm.trident.tuple.TridentTuple; public class Debug extends BaseFilter { private final String name; public Debug() { name = "DEBUG: "; } public Debug(String name) { this.name = "DEBUG(" + name + "): "; } @Override public boolean isKeep(TridentTuple tuple) { System.out.println(name + tuple.toString()); return true; } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/builtin/Equals.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.builtin; import storm.trident.operation.BaseFilter; import storm.trident.tuple.TridentTuple; public class Equals extends BaseFilter { @Override public boolean isKeep(TridentTuple tuple) { for(int i=0; i { int _n; public FirstNAgg(int n) { _n = n; } static class State { int emitted = 0; } @Override public State init(Object batchId, TridentCollector collector) { return new State(); } @Override public void aggregate(State val, TridentTuple tuple, TridentCollector collector) { if(val.emitted < _n) { collector.emit(tuple); val.emitted++; } } @Override public void complete(State val, TridentCollector collector) { } } public static class FirstNSortedAgg extends BaseAggregator { int _n; String _sortField; boolean _reverse; public FirstNSortedAgg(int n, String sortField, boolean reverse) { _n = n; _sortField = sortField; _reverse = reverse; } @Override public PriorityQueue init(Object batchId, TridentCollector collector) { return new PriorityQueue(_n, new Comparator() { @Override public int compare(TridentTuple t1, TridentTuple t2) { Comparable c1 = (Comparable) t1.getValueByField(_sortField); Comparable c2 = (Comparable) t2.getValueByField(_sortField); int ret = c1.compareTo(c2); if(_reverse) ret *= -1; return ret; } }); } @Override public void aggregate(PriorityQueue state, TridentTuple tuple, TridentCollector collector) { state.add(tuple); } @Override public void complete(PriorityQueue val, TridentCollector collector) { int total = val.size(); for(int i=0; i<_n && i < total; i++) { TridentTuple t = (TridentTuple) val.remove(); collector.emit(t); } } } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/builtin/MapGet.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.builtin; import backtype.storm.tuple.Values; import java.util.List; import storm.trident.operation.TridentCollector; import storm.trident.state.BaseQueryFunction; import storm.trident.state.map.ReadOnlyMapState; import storm.trident.tuple.TridentTuple; public class MapGet extends BaseQueryFunction { @Override public List batchRetrieve(ReadOnlyMapState map, List keys) { return map.multiGet((List) keys); } @Override public void execute(TridentTuple tuple, Object result, TridentCollector collector) { collector.emit(new Values(result)); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/builtin/Negate.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.builtin; import java.util.Map; import storm.trident.operation.Filter; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.TridentTuple; public class Negate implements Filter { Filter _delegate; public Negate(Filter delegate) { _delegate = delegate; } @Override public boolean isKeep(TridentTuple tuple) { return !_delegate.isKeep(tuple); } @Override public void prepare(Map conf, TridentOperationContext context) { _delegate.prepare(conf, context); } @Override public void cleanup() { _delegate.cleanup(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/builtin/SnapshotGet.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.builtin; import backtype.storm.tuple.Values; import java.util.ArrayList; import java.util.List; import storm.trident.operation.TridentCollector; import storm.trident.state.BaseQueryFunction; import storm.trident.state.snapshot.ReadOnlySnapshottable; import storm.trident.tuple.TridentTuple; public class SnapshotGet extends BaseQueryFunction { @Override public List batchRetrieve(ReadOnlySnapshottable state, List args) { List ret = new ArrayList(args.size()); Object snapshot = state.get(); for(int i=0; i { @Override public Number init(TridentTuple tuple) { return (Number) tuple.getValue(0); } @Override public Number combine(Number val1, Number val2) { return Numbers.add(val1, val2); } @Override public Number zero() { return 0; } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/builtin/TupleCollectionGet.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.builtin; import storm.trident.state.ITupleCollection; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import storm.trident.operation.TridentCollector; import storm.trident.state.BaseQueryFunction; import storm.trident.state.State; import storm.trident.tuple.TridentTuple; public class TupleCollectionGet extends BaseQueryFunction>> { @Override public List>> batchRetrieve(State state, List args) { List>> ret = new ArrayList(args.size()); for(int i=0; i> tuplesIterator, TridentCollector collector) { while(tuplesIterator.hasNext()) { collector.emit(tuplesIterator.next()); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/CaptureCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import java.util.ArrayList; import java.util.List; import storm.trident.operation.TridentCollector; public class CaptureCollector implements TridentCollector { public List> captured = new ArrayList(); TridentCollector _coll; public void setCollector(TridentCollector coll) { _coll = coll; } @Override public void emit(List values) { this.captured.add(values); } @Override public void reportError(Throwable t) { _coll.reportError(t); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/ChainedAggregatorImpl.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Fields; import java.util.List; import java.util.Map; import storm.trident.operation.Aggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.ComboList; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTupleView; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class ChainedAggregatorImpl implements Aggregator { Aggregator[] _aggs; ProjectionFactory[] _inputFactories; ComboList.Factory _fact; Fields[] _inputFields; public ChainedAggregatorImpl(Aggregator[] aggs, Fields[] inputFields, ComboList.Factory fact) { _aggs = aggs; _inputFields = inputFields; _fact = fact; if(_aggs.length!=_inputFields.length) { throw new IllegalArgumentException("Require input fields for each aggregator"); } } public void prepare(Map conf, TridentOperationContext context) { _inputFactories = new ProjectionFactory[_inputFields.length]; for(int i=0; i<_inputFields.length; i++) { _inputFactories[i] = context.makeProjectionFactory(_inputFields[i]); _aggs[i].prepare(conf, new TridentOperationContext(context, _inputFactories[i])); } } public ChainedResult init(Object batchId, TridentCollector collector) { ChainedResult initted = new ChainedResult(collector, _aggs.length); for(int i=0; i<_aggs.length; i++) { initted.objs[i] = _aggs[i].init(batchId, initted.collectors[i]); } return initted; } public void aggregate(ChainedResult val, TridentTuple tuple, TridentCollector collector) { val.setFollowThroughCollector(collector); for(int i=0; i<_aggs.length; i++) { TridentTuple projected = _inputFactories[i].create((TridentTupleView) tuple); _aggs[i].aggregate(val.objs[i], projected, val.collectors[i]); } } public void complete(ChainedResult val, TridentCollector collector) { val.setFollowThroughCollector(collector); for(int i=0; i<_aggs.length; i++) { _aggs[i].complete(val.objs[i], val.collectors[i]); } if(_aggs.length > 1) { // otherwise, tuples were emitted directly int[] indices = new int[val.collectors.length]; for(int i=0; i= capturer.captured.size()) { indices[j] = 0; return increment(lengths, indices, j-1); } return true; } public void cleanup() { for(Aggregator a: _aggs) { a.cleanup(); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/ChainedResult.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import org.apache.commons.lang.builder.ToStringBuilder; import storm.trident.operation.TridentCollector; //for ChainedAggregator public class ChainedResult { Object[] objs; TridentCollector[] collectors; public ChainedResult(TridentCollector collector, int size) { objs = new Object[size]; collectors = new TridentCollector[size]; for(int i=0; i1) { for(TridentCollector c: collectors) { ((CaptureCollector) c).setCollector(collector); } } } @Override public String toString() { return ToStringBuilder.reflectionToString(objs); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/CombinerAggStateUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Values; import java.util.List; import java.util.Map; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.state.CombinerValueUpdater; import storm.trident.state.StateUpdater; import storm.trident.state.snapshot.Snapshottable; import storm.trident.tuple.TridentTuple; public class CombinerAggStateUpdater implements StateUpdater { CombinerAggregator _agg; public CombinerAggStateUpdater(CombinerAggregator agg) { _agg = agg; } @Override public void updateState(Snapshottable state, List tuples, TridentCollector collector) { if(tuples.size()!=1) { throw new IllegalArgumentException("Combiner state updater should receive a single tuple. Received: " + tuples.toString()); } Object newVal = state.update(new CombinerValueUpdater(_agg, tuples.get(0).getValue(0))); collector.emit(new Values(newVal)); } @Override public void prepare(Map conf, TridentOperationContext context) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Values; import java.util.Map; import storm.trident.operation.Aggregator; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.TridentTuple; public class CombinerAggregatorCombineImpl implements Aggregator { CombinerAggregator _agg; public CombinerAggregatorCombineImpl(CombinerAggregator agg) { _agg = agg; } public void prepare(Map conf, TridentOperationContext context) { } public Result init(Object batchId, TridentCollector collector) { Result ret = new Result(); ret.obj = _agg.zero(); return ret; } public void aggregate(Result val, TridentTuple tuple, TridentCollector collector) { Object v = tuple.getValue(0); if(val.obj==null) { val.obj = v; } else { val.obj = _agg.combine(val.obj, v); } } public void complete(Result val, TridentCollector collector) { collector.emit(new Values(val.obj)); } public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/CombinerAggregatorInitImpl.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Values; import java.util.Map; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.Function; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.TridentTuple; public class CombinerAggregatorInitImpl implements Function { CombinerAggregator _agg; public CombinerAggregatorInitImpl(CombinerAggregator agg) { _agg = agg; } @Override public void execute(TridentTuple tuple, TridentCollector collector) { collector.emit(new Values(_agg.init(tuple))); } @Override public void prepare(Map conf, TridentOperationContext context) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/FilterExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import java.util.Map; import storm.trident.operation.Filter; import storm.trident.operation.Function; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.TridentTuple; // works by emitting null to the collector. since the planner knows this is an ADD node with // no new output fields, it just passes the tuple forward public class FilterExecutor implements Function { Filter _filter; public FilterExecutor(Filter filter) { _filter = filter; } @Override public void execute(TridentTuple tuple, TridentCollector collector) { if(_filter.isKeep(tuple)) { collector.emit(null); } } @Override public void prepare(Map conf, TridentOperationContext context) { _filter.prepare(conf, context); } @Override public void cleanup() { _filter.cleanup(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/GlobalBatchToPartition.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; public class GlobalBatchToPartition implements SingleEmitAggregator.BatchToPartition { @Override public int partitionIndex(Object batchId, int numPartitions) { // TODO: take away knowledge of storm's internals here return 0; } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/GroupCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import java.util.List; import storm.trident.operation.TridentCollector; import storm.trident.tuple.ComboList; public class GroupCollector implements TridentCollector { public List currGroup; ComboList.Factory _factory; TridentCollector _collector; public GroupCollector(TridentCollector collector, ComboList.Factory factory) { _factory = factory; _collector = collector; } @Override public void emit(List values) { List[] delegates = new List[2]; delegates[0] = currGroup; delegates[1] = values; _collector.emit(_factory.create(delegates)); } @Override public void reportError(Throwable t) { _collector.reportError(t); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/GroupedAggregator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Fields; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import storm.trident.operation.Aggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.ComboList; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTupleView; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class GroupedAggregator implements Aggregator { ProjectionFactory _groupFactory; ProjectionFactory _inputFactory; Aggregator _agg; ComboList.Factory _fact; Fields _inFields; Fields _groupFields; public GroupedAggregator(Aggregator agg, Fields group, Fields input, int outSize) { _groupFields = group; _inFields = input; _agg = agg; int[] sizes = new int[2]; sizes[0] = _groupFields.size(); sizes[1] = outSize; _fact = new ComboList.Factory(sizes); } @Override public void prepare(Map conf, TridentOperationContext context) { _inputFactory = context.makeProjectionFactory(_inFields); _groupFactory = context.makeProjectionFactory(_groupFields); _agg.prepare(conf, new TridentOperationContext(context, _inputFactory)); } @Override public Object[] init(Object batchId, TridentCollector collector) { return new Object[] {new GroupCollector(collector, _fact), new HashMap(), batchId}; } @Override public void aggregate(Object[] arr, TridentTuple tuple, TridentCollector collector) { GroupCollector groupColl = (GroupCollector) arr[0]; Map val = (Map) arr[1]; TridentTuple group = _groupFactory.create((TridentTupleView) tuple); TridentTuple input = _inputFactory.create((TridentTupleView) tuple); Object curr; if(!val.containsKey(group)) { curr = _agg.init(arr[2], groupColl); val.put((List) group, curr); } else { curr = val.get(group); } groupColl.currGroup = group; _agg.aggregate(curr, input, groupColl); } @Override public void complete(Object[] arr, TridentCollector collector) { Map val = (Map) arr[1]; GroupCollector groupColl = (GroupCollector) arr[0]; for(Entry e: val.entrySet()) { groupColl.currGroup = e.getKey(); _agg.complete(e.getValue(), groupColl); } } @Override public void cleanup() { _agg.cleanup(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/GroupedMultiReducerExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import storm.trident.operation.GroupedMultiReducer; import storm.trident.operation.MultiReducer; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentMultiReducerContext; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class GroupedMultiReducerExecutor implements MultiReducer> { GroupedMultiReducer _reducer; List _groupFields; List _inputFields; List _groupFactories = new ArrayList(); List _inputFactories = new ArrayList(); public GroupedMultiReducerExecutor(GroupedMultiReducer reducer, List groupFields, List inputFields) { if(inputFields.size()!=groupFields.size()) { throw new IllegalArgumentException("Multireducer groupFields and inputFields must be the same size"); } _groupFields = groupFields; _inputFields = inputFields; _reducer = reducer; } @Override public void prepare(Map conf, TridentMultiReducerContext context) { for(int i=0; i<_groupFields.size(); i++) { _groupFactories.add(context.makeProjectionFactory(i, _groupFields.get(i))); _inputFactories.add(context.makeProjectionFactory(i, _inputFields.get(i))); } _reducer.prepare(conf, new TridentMultiReducerContext((List) _inputFactories)); } @Override public Map init(TridentCollector collector) { return new HashMap(); } @Override public void execute(Map state, int streamIndex, TridentTuple full, TridentCollector collector) { ProjectionFactory groupFactory = _groupFactories.get(streamIndex); ProjectionFactory inputFactory = _inputFactories.get(streamIndex); TridentTuple group = groupFactory.create(full); TridentTuple input = inputFactory.create(full); Object curr; if(!state.containsKey(group)) { curr = _reducer.init(collector, group); state.put(group, curr); } else { curr = state.get(group); } _reducer.execute(curr, streamIndex, group, input, collector); } @Override public void complete(Map state, TridentCollector collector) { for(Map.Entry e: state.entrySet()) { TridentTuple group = (TridentTuple) e.getKey(); Object val = e.getValue(); _reducer.complete(val, group, collector); } } @Override public void cleanup() { _reducer.cleanup(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/IdentityMultiReducer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import java.util.Map; import storm.trident.operation.MultiReducer; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentMultiReducerContext; import storm.trident.tuple.TridentTuple; public class IdentityMultiReducer implements MultiReducer { @Override public void prepare(Map conf, TridentMultiReducerContext context) { } @Override public Object init(TridentCollector collector) { return null; } @Override public void execute(Object state, int streamIndex, TridentTuple input, TridentCollector collector) { collector.emit(input); } @Override public void complete(Object state, TridentCollector collector) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/IndexHashBatchToPartition.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import storm.trident.partition.IndexHashGrouping; public class IndexHashBatchToPartition implements SingleEmitAggregator.BatchToPartition { @Override public int partitionIndex(Object batchId, int numPartitions) { return IndexHashGrouping.objectToIndex(batchId, numPartitions); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/JoinerMultiReducer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.List; import java.util.Map; import storm.trident.JoinType; import storm.trident.operation.GroupedMultiReducer; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentMultiReducerContext; import storm.trident.operation.impl.JoinerMultiReducer.JoinState; import storm.trident.tuple.ComboList; import storm.trident.tuple.TridentTuple; public class JoinerMultiReducer implements GroupedMultiReducer { List _types; List _sideFields; int _numGroupFields; ComboList.Factory _factory; public JoinerMultiReducer(List types, int numGroupFields, List sides) { _types = types; _sideFields = sides; _numGroupFields = numGroupFields; } @Override public void prepare(Map conf, TridentMultiReducerContext context) { int[] sizes = new int[_sideFields.size() + 1]; sizes[0] = _numGroupFields; for(int i=0; i<_sideFields.size(); i++) { sizes[i+1] = _sideFields.get(i).size(); } _factory = new ComboList.Factory(sizes); } @Override public JoinState init(TridentCollector collector, TridentTuple group) { return new JoinState(_types.size(), group); } @Override public void execute(JoinState state, int streamIndex, TridentTuple group, TridentTuple input, TridentCollector collector) { //TODO: do the inner join incrementally, emitting the cross join with this tuple, against all other sides //TODO: only do cross join if at least one tuple in each side List side = state.sides[streamIndex]; if(side.isEmpty()) { state.numSidesReceived++; } side.add(input); if(state.numSidesReceived == state.sides.length) { emitCrossJoin(state, collector, streamIndex, input); } } @Override public void complete(JoinState state, TridentTuple group, TridentCollector collector) { List[] sides = state.sides; boolean wasEmpty = state.numSidesReceived < sides.length; for(int i=0; i makeNullList(int size) { List ret = new ArrayList(size); for(int i=0; i[] sides = state.sides; int[] indices = state.indices; for(int i=0; i= lengths[j].size()) { indices[j] = 0; return increment(lengths, indices, j-1, overrideIndex); } return true; } public static class JoinState { List[] sides; int numSidesReceived = 0; int[] indices; TridentTuple group; public JoinState(int numSides, TridentTuple group) { sides = new List[numSides]; indices = new int[numSides]; this.group = group; for(int i=0; i(); } } } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/ReducerAggStateUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Values; import java.util.List; import java.util.Map; import storm.trident.operation.ReducerAggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.state.ReducerValueUpdater; import storm.trident.state.StateUpdater; import storm.trident.state.snapshot.Snapshottable; import storm.trident.tuple.TridentTuple; public class ReducerAggStateUpdater implements StateUpdater { ReducerAggregator _agg; public ReducerAggStateUpdater(ReducerAggregator agg) { _agg = agg; } @Override public void updateState(Snapshottable state, List tuples, TridentCollector collector) { Object newVal = state.update(new ReducerValueUpdater(_agg, tuples)); collector.emit(new Values(newVal)); } @Override public void prepare(Map conf, TridentOperationContext context) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/ReducerAggregatorImpl.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import backtype.storm.tuple.Values; import java.util.Map; import storm.trident.operation.Aggregator; import storm.trident.operation.ReducerAggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.TridentTuple; public class ReducerAggregatorImpl implements Aggregator { ReducerAggregator _agg; public ReducerAggregatorImpl(ReducerAggregator agg) { _agg = agg; } public void prepare(Map conf, TridentOperationContext context) { } public Result init(Object batchId, TridentCollector collector) { Result ret = new Result(); ret.obj = _agg.init(); return ret; } public void aggregate(Result val, TridentTuple tuple, TridentCollector collector) { val.obj = _agg.reduce(val.obj, tuple); } public void complete(Result val, TridentCollector collector) { collector.emit(new Values(val.obj)); } public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/Result.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; public class Result { public Object obj; @Override public String toString() { return "" + obj; } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/SingleEmitAggregator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import java.io.Serializable; import java.util.Map; import storm.trident.operation.Aggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.operation.impl.SingleEmitAggregator.SingleEmitState; import storm.trident.tuple.TridentTuple; public class SingleEmitAggregator implements Aggregator { public static interface BatchToPartition extends Serializable { int partitionIndex(Object batchId, int numPartitions); } static class SingleEmitState { boolean received = false; Object state; Object batchId; public SingleEmitState(Object batchId) { this.batchId = batchId; } } Aggregator _agg; BatchToPartition _batchToPartition; public SingleEmitAggregator(Aggregator agg, BatchToPartition batchToPartition) { _agg = agg; _batchToPartition = batchToPartition; } @Override public SingleEmitState init(Object batchId, TridentCollector collector) { return new SingleEmitState(batchId); } @Override public void aggregate(SingleEmitState val, TridentTuple tuple, TridentCollector collector) { if(!val.received) { val.state = _agg.init(val.batchId, collector); val.received = true; } _agg.aggregate(val.state, tuple, collector); } @Override public void complete(SingleEmitState val, TridentCollector collector) { if(!val.received) { if(this.myPartitionIndex == _batchToPartition.partitionIndex(val.batchId, this.totalPartitions)) { val.state = _agg.init(val.batchId, collector); _agg.complete(val.state, collector); } } else { _agg.complete(val.state, collector); } } int myPartitionIndex; int totalPartitions; @Override public void prepare(Map conf, TridentOperationContext context) { _agg.prepare(conf, context); this.myPartitionIndex = context.getPartitionIndex(); this.totalPartitions = context.numPartitions(); } @Override public void cleanup() { _agg.cleanup(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/operation/impl/TrueFilter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.operation.impl; import java.util.Map; import storm.trident.operation.Filter; import storm.trident.operation.TridentOperationContext; import storm.trident.tuple.TridentTuple; public class TrueFilter implements Filter { @Override public boolean isKeep(TridentTuple tuple) { return true; } @Override public void prepare(Map conf, TridentOperationContext context) { } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/partition/GlobalGrouping.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.partition; import backtype.storm.generated.GlobalStreamId; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.task.WorkerTopologyContext; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; public class GlobalGrouping implements CustomStreamGrouping { List target; @Override public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targets) { List sorted = new ArrayList(targets); Collections.sort(sorted); target = Arrays.asList(sorted.get(0)); } @Override public List chooseTasks(int i, List list) { return target; } } ================================================ FILE: storm-core/src/jvm/storm/trident/partition/IdentityGrouping.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.partition; import backtype.storm.generated.GlobalStreamId; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.task.WorkerTopologyContext; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; public class IdentityGrouping implements CustomStreamGrouping { List ret = new ArrayList(); Map> _precomputed = new HashMap(); @Override public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List tasks) { List sourceTasks = new ArrayList(context.getComponentTasks(stream.get_componentId())); Collections.sort(sourceTasks); if(sourceTasks.size()!=tasks.size()) { throw new RuntimeException("Can only do an identity grouping when source and target have same number of tasks"); } tasks = new ArrayList(tasks); Collections.sort(tasks); for(int i=0; i chooseTasks(int task, List values) { List ret = _precomputed.get(task); if(ret==null) { throw new RuntimeException("Tuple emitted by task that's not part of this component. Should be impossible"); } return ret; } } ================================================ FILE: storm-core/src/jvm/storm/trident/partition/IndexHashGrouping.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.partition; import backtype.storm.generated.GlobalStreamId; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.task.WorkerTopologyContext; import java.util.Arrays; import java.util.List; public class IndexHashGrouping implements CustomStreamGrouping { public static int objectToIndex(Object val, int numPartitions) { if(val==null) return 0; else { return Math.abs(val.hashCode()) % numPartitions; } } int _index; List _targets; public IndexHashGrouping(int index) { _index = index; } @Override public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) { _targets = targetTasks; } @Override public List chooseTasks(int fromTask, List values) { int i = objectToIndex(values.get(_index), _targets.size()); return Arrays.asList(_targets.get(i)); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/BridgeReceiver.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import backtype.storm.coordination.BatchOutputCollector; import storm.trident.tuple.ConsList; import storm.trident.tuple.TridentTuple; public class BridgeReceiver implements TupleReceiver { BatchOutputCollector _collector; public BridgeReceiver(BatchOutputCollector collector) { _collector = collector; } @Override public void execute(ProcessorContext context, String streamId, TridentTuple tuple) { _collector.emit(streamId, new ConsList(context.batchId, tuple)); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/Node.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import backtype.storm.tuple.Fields; import java.io.Serializable; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang.builder.ToStringBuilder; import org.apache.commons.lang.builder.ToStringStyle; public class Node implements Serializable { private static AtomicInteger INDEX = new AtomicInteger(0); private String nodeId; public String name = null; public Fields allOutputFields; public String streamId; public Integer parallelismHint = null; public NodeStateInfo stateInfo = null; public int creationIndex; public Node(String streamId, String name, Fields allOutputFields) { this.nodeId = UUID.randomUUID().toString(); this.allOutputFields = allOutputFields; this.streamId = streamId; this.name = name; this.creationIndex = INDEX.incrementAndGet(); } @Override public boolean equals(Object o) { return nodeId.equals(((Node) o).nodeId); } @Override public int hashCode() { return nodeId.hashCode(); } @Override public String toString() { return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/NodeStateInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import java.io.Serializable; import storm.trident.state.StateSpec; public class NodeStateInfo implements Serializable { public String id; public StateSpec spec; public NodeStateInfo(String id, StateSpec spec) { this.id = id; this.spec = spec; } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/PartitionNode.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import backtype.storm.generated.Grouping; import backtype.storm.tuple.Fields; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.util.ArrayList; import java.util.List; import storm.trident.util.TridentUtils; public class PartitionNode extends Node { public transient Grouping thriftGrouping; //has the streamid/outputFields of the node it's doing the partitioning on public PartitionNode(String streamId, String name, Fields allOutputFields, Grouping grouping) { super(streamId, name, allOutputFields); this.thriftGrouping = grouping; } private void writeObject(ObjectOutputStream oos) throws IOException { oos.defaultWriteObject(); byte[] ser = TridentUtils.thriftSerialize(thriftGrouping); oos.writeInt(ser.length); oos.write(ser); } private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { ois.defaultReadObject(); byte[] ser = new byte[ois.readInt()]; ois.readFully(ser); this.thriftGrouping = TridentUtils.thriftDeserialize(Grouping.class, ser); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/ProcessorContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; public class ProcessorContext { public Object batchId; public Object[] state; public ProcessorContext(Object batchId, Object[] state) { this.batchId = batchId; this.state = state; } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/ProcessorNode.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import backtype.storm.tuple.Fields; public class ProcessorNode extends Node { public boolean committer; // for partitionpersist public TridentProcessor processor; public Fields selfOutFields; public ProcessorNode(String streamId, String name, Fields allOutputFields, Fields selfOutFields, TridentProcessor processor) { super(streamId, name, allOutputFields); this.processor = processor; this.selfOutFields = selfOutFields; } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/SpoutNode.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import backtype.storm.tuple.Fields; public class SpoutNode extends Node { public static enum SpoutType { DRPC, BATCH } public Object spout; public String txId; //where state is stored in zookeeper (only for batch spout types) public SpoutType type; public SpoutNode(String streamId, Fields allOutputFields, String txid, Object spout, SpoutType type) { super(streamId, null, allOutputFields); this.txId = txid; this.spout = spout; this.type = type; } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/SubtopologyBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.generated.GlobalStreamId; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import org.jgrapht.DirectedGraph; import org.jgrapht.graph.DirectedSubgraph; import org.jgrapht.traverse.TopologicalOrderIterator; import storm.trident.planner.processor.TridentContext; import storm.trident.state.State; import storm.trident.topology.BatchInfo; import storm.trident.topology.ITridentBatchBolt; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView.ProjectionFactory; import storm.trident.tuple.TridentTupleView.RootFactory; import storm.trident.util.TridentUtils; // TODO: parameterizing it like this with everything might be a high deserialization cost if there's lots of tasks? // TODO: memory problems? // TODO: can avoid these problems by adding a boltfactory abstraction, so that boltfactory is deserialized once // bolt factory -> returns coordinatedbolt per task, but deserializes the batch bolt one time and caches public class SubtopologyBolt implements ITridentBatchBolt { DirectedGraph _graph; Set _nodes; Map _roots = new HashMap(); Map _outputFactories = new HashMap(); Map> _myTopologicallyOrdered = new HashMap(); Map _batchGroups; //given processornodes and static state nodes public SubtopologyBolt(DirectedGraph graph, Set nodes, Map batchGroups) { _nodes = nodes; _graph = graph; _batchGroups = batchGroups; } @Override public void prepare(Map conf, TopologyContext context, BatchOutputCollector batchCollector) { int thisComponentNumTasks = context.getComponentTasks(context.getThisComponentId()).size(); for(Node n: _nodes) { if(n.stateInfo!=null) { State s = n.stateInfo.spec.stateFactory.makeState(conf, context, context.getThisTaskIndex(), thisComponentNumTasks); context.setTaskData(n.stateInfo.id, s); } } DirectedSubgraph subgraph = new DirectedSubgraph(_graph, _nodes, null); TopologicalOrderIterator it = new TopologicalOrderIterator(subgraph); int stateIndex = 0; while(it.hasNext()) { Node n = (Node) it.next(); if(n instanceof ProcessorNode) { ProcessorNode pn = (ProcessorNode) n; String batchGroup = _batchGroups.get(n); if(!_myTopologicallyOrdered.containsKey(batchGroup)) { _myTopologicallyOrdered.put(batchGroup, new ArrayList()); } _myTopologicallyOrdered.get(batchGroup).add(pn.processor); List parentStreams = new ArrayList(); List parentFactories = new ArrayList(); for(Node p: TridentUtils.getParents(_graph, n)) { parentStreams.add(p.streamId); if(_nodes.contains(p)) { parentFactories.add(_outputFactories.get(p)); } else { if(!_roots.containsKey(p.streamId)) { _roots.put(p.streamId, new InitialReceiver(p.streamId, getSourceOutputFields(context, p.streamId))); } _roots.get(p.streamId).addReceiver(pn.processor); parentFactories.add(_roots.get(p.streamId).getOutputFactory()); } } List targets = new ArrayList(); boolean outgoingNode = false; for(Node cn: TridentUtils.getChildren(_graph, n)) { if(_nodes.contains(cn)) { targets.add(((ProcessorNode) cn).processor); } else { outgoingNode = true; } } if(outgoingNode) { targets.add(new BridgeReceiver(batchCollector)); } TridentContext triContext = new TridentContext( pn.selfOutFields, parentFactories, parentStreams, targets, pn.streamId, stateIndex, batchCollector ); pn.processor.prepare(conf, context, triContext); _outputFactories.put(n, pn.processor.getOutputFactory()); } stateIndex++; } // TODO: get prepared one time into executor data... need to avoid the ser/deser // for each task (probably need storm to support boltfactory) } private Fields getSourceOutputFields(TopologyContext context, String sourceStream) { for(GlobalStreamId g: context.getThisSources().keySet()) { if(g.get_streamId().equals(sourceStream)) { return context.getComponentOutputFields(g); } } throw new RuntimeException("Could not find fields for source stream " + sourceStream); } @Override public void execute(BatchInfo batchInfo, Tuple tuple) { String sourceStream = tuple.getSourceStreamId(); InitialReceiver ir = _roots.get(sourceStream); if(ir==null) { throw new RuntimeException("Received unexpected tuple " + tuple.toString()); } ir.receive((ProcessorContext) batchInfo.state, tuple); } @Override public void finishBatch(BatchInfo batchInfo) { for(TridentProcessor p: _myTopologicallyOrdered.get(batchInfo.batchGroup)) { p.finishBatch((ProcessorContext) batchInfo.state); } } @Override public Object initBatchState(String batchGroup, Object batchId) { ProcessorContext ret = new ProcessorContext(batchId, new Object[_nodes.size()]); for(TridentProcessor p: _myTopologicallyOrdered.get(batchGroup)) { p.startBatch(ret); } return ret; } @Override public void cleanup() { for(String bg: _myTopologicallyOrdered.keySet()) { for(TridentProcessor p: _myTopologicallyOrdered.get(bg)) { p.cleanup(); } } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { for(Node n: _nodes) { declarer.declareStream(n.streamId, TridentUtils.fieldsConcat(new Fields("$batchId"), n.allOutputFields)); } } @Override public Map getComponentConfiguration() { return null; } protected class InitialReceiver { List _receivers = new ArrayList(); RootFactory _factory; ProjectionFactory _project; String _stream; public InitialReceiver(String stream, Fields allFields) { // TODO: don't want to project for non-batch bolts...??? // how to distinguish "batch" streams from non-batch streams? _stream = stream; _factory = new RootFactory(allFields); List projected = new ArrayList(allFields.toList()); projected.remove(0); _project = new ProjectionFactory(_factory, new Fields(projected)); } public void receive(ProcessorContext context, Tuple tuple) { TridentTuple t = _project.create(_factory.create(tuple)); for(TridentProcessor r: _receivers) { r.execute(context, _stream, t); } } public void addReceiver(TridentProcessor p) { _receivers.add(p); } public Factory getOutputFactory() { return _project; } } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/TridentProcessor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import backtype.storm.task.TopologyContext; import java.io.Serializable; import java.util.Map; import storm.trident.planner.processor.TridentContext; import storm.trident.tuple.TridentTuple.Factory; public interface TridentProcessor extends Serializable, TupleReceiver { // imperative that don't emit any tuples from here, since output factory cannot be gotten until // preparation is done, therefore, receivers won't be ready to receive tuples yet // can't emit tuples from here anyway, since it's not within a batch context (which is only // startBatch, execute, and finishBatch void prepare(Map conf, TopologyContext context, TridentContext tridentContext); void cleanup(); void startBatch(ProcessorContext processorContext); void finishBatch(ProcessorContext processorContext); Factory getOutputFactory(); } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/TupleReceiver.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner; import storm.trident.tuple.TridentTuple; public interface TupleReceiver { //streaId indicates where tuple came from void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple); } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/AggregateProcessor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.List; import java.util.Map; import storm.trident.operation.Aggregator; import storm.trident.operation.TridentOperationContext; import storm.trident.planner.ProcessorContext; import storm.trident.planner.TridentProcessor; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class AggregateProcessor implements TridentProcessor { Aggregator _agg; TridentContext _context; FreshCollector _collector; Fields _inputFields; ProjectionFactory _projection; public AggregateProcessor(Fields inputFields, Aggregator agg) { _agg = agg; _inputFields = inputFields; } @Override public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { List parents = tridentContext.getParentTupleFactories(); if(parents.size()!=1) { throw new RuntimeException("Aggregate operation can only have one parent"); } _context = tridentContext; _collector = new FreshCollector(tridentContext); _projection = new ProjectionFactory(parents.get(0), _inputFields); _agg.prepare(conf, new TridentOperationContext(context, _projection)); } @Override public void cleanup() { _agg.cleanup(); } @Override public void startBatch(ProcessorContext processorContext) { _collector.setContext(processorContext); processorContext.state[_context.getStateIndex()] = _agg.init(processorContext.batchId, _collector); } @Override public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { _collector.setContext(processorContext); _agg.aggregate(processorContext.state[_context.getStateIndex()], _projection.create(tuple), _collector); } @Override public void finishBatch(ProcessorContext processorContext) { _collector.setContext(processorContext); _agg.complete(processorContext.state[_context.getStateIndex()], _collector); } @Override public Factory getOutputFactory() { return _collector.getOutputFactory(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/AppendCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import java.util.List; import storm.trident.operation.TridentCollector; import storm.trident.planner.ProcessorContext; import storm.trident.planner.TupleReceiver; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView; import storm.trident.tuple.TridentTupleView.OperationOutputFactory; public class AppendCollector implements TridentCollector { OperationOutputFactory _factory; TridentContext _triContext; TridentTuple tuple; ProcessorContext context; public AppendCollector(TridentContext context) { _triContext = context; _factory = new OperationOutputFactory(context.getParentTupleFactories().get(0), context.getSelfOutputFields()); } public void setContext(ProcessorContext pc, TridentTuple t) { this.context = pc; this.tuple = t; } @Override public void emit(List values) { TridentTuple toEmit = _factory.create((TridentTupleView) tuple, values); for(TupleReceiver r: _triContext.getReceivers()) { r.execute(context, _triContext.getOutStreamId(), toEmit); } } @Override public void reportError(Throwable t) { _triContext.getDelegateCollector().reportError(t); } public Factory getOutputFactory() { return _factory; } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/EachProcessor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.List; import java.util.Map; import storm.trident.operation.Function; import storm.trident.operation.TridentOperationContext; import storm.trident.planner.ProcessorContext; import storm.trident.planner.TridentProcessor; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class EachProcessor implements TridentProcessor { Function _function; TridentContext _context; AppendCollector _collector; Fields _inputFields; ProjectionFactory _projection; public EachProcessor(Fields inputFields, Function function) { _function = function; _inputFields = inputFields; } @Override public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { List parents = tridentContext.getParentTupleFactories(); if(parents.size()!=1) { throw new RuntimeException("Each operation can only have one parent"); } _context = tridentContext; _collector = new AppendCollector(tridentContext); _projection = new ProjectionFactory(parents.get(0), _inputFields); _function.prepare(conf, new TridentOperationContext(context, _projection)); } @Override public void cleanup() { _function.cleanup(); } @Override public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { _collector.setContext(processorContext, tuple); _function.execute(_projection.create(tuple), _collector); } @Override public void startBatch(ProcessorContext processorContext) { } @Override public void finishBatch(ProcessorContext processorContext) { } @Override public Factory getOutputFactory() { return _collector.getOutputFactory(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/FreshCollector.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import java.util.List; import storm.trident.operation.TridentCollector; import storm.trident.planner.ProcessorContext; import storm.trident.planner.TupleReceiver; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView.FreshOutputFactory; public class FreshCollector implements TridentCollector { FreshOutputFactory _factory; TridentContext _triContext; ProcessorContext context; public FreshCollector(TridentContext context) { _triContext = context; _factory = new FreshOutputFactory(context.getSelfOutputFields()); } public void setContext(ProcessorContext pc) { this.context = pc; } @Override public void emit(List values) { TridentTuple toEmit = _factory.create(values); for(TupleReceiver r: _triContext.getReceivers()) { r.execute(context, _triContext.getOutStreamId(), toEmit); } } @Override public void reportError(Throwable t) { _triContext.getDelegateCollector().reportError(t); } public Factory getOutputFactory() { return _factory; } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/MultiReducerProcessor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import storm.trident.operation.MultiReducer; import storm.trident.operation.TridentMultiReducerContext; import storm.trident.planner.ProcessorContext; import storm.trident.planner.TridentProcessor; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class MultiReducerProcessor implements TridentProcessor { MultiReducer _reducer; TridentContext _context; Map _streamToIndex; List _projectFields; ProjectionFactory[] _projectionFactories; FreshCollector _collector; public MultiReducerProcessor(List inputFields, MultiReducer reducer) { _reducer = reducer; _projectFields = inputFields; } @Override public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { List parents = tridentContext.getParentTupleFactories(); _context = tridentContext; _streamToIndex = new HashMap(); List parentStreams = tridentContext.getParentStreams(); for(int i=0; i parents = tridentContext.getParentTupleFactories(); if(parents.size()!=1) { throw new RuntimeException("Partition persist operation can only have one parent"); } _context = tridentContext; _state = (State) context.getTaskData(_stateId); _projection = new ProjectionFactory(parents.get(0), _inputFields); _collector = new FreshCollector(tridentContext); _updater.prepare(conf, new TridentOperationContext(context, _projection)); } @Override public void cleanup() { _updater.cleanup(); } @Override public void startBatch(ProcessorContext processorContext) { processorContext.state[_context.getStateIndex()] = new ArrayList(); } @Override public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { ((List) processorContext.state[_context.getStateIndex()]).add(_projection.create(tuple)); } @Override public void finishBatch(ProcessorContext processorContext) { _collector.setContext(processorContext); Object batchId = processorContext.batchId; // since this processor type is a committer, this occurs in the commit phase List buffer = (List) processorContext.state[_context.getStateIndex()]; // don't update unless there are tuples // this helps out with things like global partition persist, where multiple tasks may still // exist for this processor. Only want the global one to do anything // this is also a helpful optimization that state implementations don't need to manually do if(buffer.size() > 0) { Long txid = null; // this is to support things like persisting off of drpc stream, which is inherently unreliable // and won't have a tx attempt if(batchId instanceof TransactionAttempt) { txid = ((TransactionAttempt) batchId).getTransactionId(); } _state.beginCommit(txid); _updater.updateState(_state, buffer, _collector); _state.commit(txid); } } @Override public Factory getOutputFactory() { return _collector.getOutputFactory(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/ProjectedProcessor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.Map; import storm.trident.planner.ProcessorContext; import storm.trident.planner.TridentProcessor; import storm.trident.planner.TupleReceiver; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class ProjectedProcessor implements TridentProcessor { Fields _projectFields; ProjectionFactory _factory; TridentContext _context; public ProjectedProcessor(Fields projectFields) { _projectFields = projectFields; } @Override public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { if(tridentContext.getParentTupleFactories().size()!=1) { throw new RuntimeException("Projection processor can only have one parent"); } _context = tridentContext; _factory = new ProjectionFactory(tridentContext.getParentTupleFactories().get(0), _projectFields); } @Override public void cleanup() { } @Override public void startBatch(ProcessorContext processorContext) { } @Override public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { TridentTuple toEmit = _factory.create(tuple); for(TupleReceiver r: _context.getReceivers()) { r.execute(processorContext, _context.getOutStreamId(), toEmit); } } @Override public void finishBatch(ProcessorContext processorContext) { } @Override public Factory getOutputFactory() { return _factory; } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/StateQueryProcessor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.List; import java.util.Map; import storm.trident.operation.TridentOperationContext; import storm.trident.planner.ProcessorContext; import storm.trident.planner.TridentProcessor; import storm.trident.state.QueryFunction; import storm.trident.state.State; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTuple.Factory; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class StateQueryProcessor implements TridentProcessor { QueryFunction _function; State _state; String _stateId; TridentContext _context; Fields _inputFields; ProjectionFactory _projection; AppendCollector _collector; public StateQueryProcessor(String stateId, Fields inputFields, QueryFunction function) { _stateId = stateId; _function = function; _inputFields = inputFields; } @Override public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { List parents = tridentContext.getParentTupleFactories(); if(parents.size()!=1) { throw new RuntimeException("State query operation can only have one parent"); } _context = tridentContext; _state = (State) context.getTaskData(_stateId); _projection = new ProjectionFactory(parents.get(0), _inputFields); _collector = new AppendCollector(tridentContext); _function.prepare(conf, new TridentOperationContext(context, _projection)); } @Override public void cleanup() { _function.cleanup(); } @Override public void startBatch(ProcessorContext processorContext) { processorContext.state[_context.getStateIndex()] = new BatchState(); } @Override public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { BatchState state = (BatchState) processorContext.state[_context.getStateIndex()]; state.tuples.add(tuple); state.args.add(_projection.create(tuple)); } @Override public void finishBatch(ProcessorContext processorContext) { BatchState state = (BatchState) processorContext.state[_context.getStateIndex()]; if(!state.tuples.isEmpty()) { List results = _function.batchRetrieve(_state, state.args); if(results.size()!=state.tuples.size()) { throw new RuntimeException("Results size is different than argument size: " + results.size() + " vs " + state.tuples.size()); } for(int i=0; i tuples = new ArrayList(); public List args = new ArrayList(); } @Override public Factory getOutputFactory() { return _collector.getOutputFactory(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/planner/processor/TridentContext.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.planner.processor; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.tuple.Fields; import java.util.List; import storm.trident.planner.TupleReceiver; import storm.trident.tuple.TridentTuple.Factory; public class TridentContext { Fields selfFields; List parentFactories; List parentStreams; List receivers; String outStreamId; int stateIndex; BatchOutputCollector collector; public TridentContext(Fields selfFields, List parentFactories, List parentStreams, List receivers, String outStreamId, int stateIndex, BatchOutputCollector collector) { this.selfFields = selfFields; this.parentFactories = parentFactories; this.parentStreams = parentStreams; this.receivers = receivers; this.outStreamId = outStreamId; this.stateIndex = stateIndex; this.collector = collector; } public List getParentTupleFactories() { return parentFactories; } public Fields getSelfOutputFields() { return selfFields; } public List getParentStreams() { return parentStreams; } public List getReceivers() { return receivers; } public String getOutStreamId() { return outStreamId; } public int getStateIndex() { return stateIndex; } //for reporting errors public BatchOutputCollector getDelegateCollector() { return collector; } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/BatchSpoutExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import storm.trident.topology.TransactionAttempt; import backtype.storm.tuple.Fields; import java.util.Map; import storm.trident.operation.TridentCollector; public class BatchSpoutExecutor implements ITridentSpout { public static class EmptyCoordinator implements BatchCoordinator { @Override public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) { return null; } @Override public void close() { } @Override public void success(long txid) { } @Override public boolean isReady(long txid) { return true; } } public class BatchSpoutEmitter implements Emitter { @Override public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { _spout.emitBatch(tx.getTransactionId(), collector); } @Override public void success(TransactionAttempt tx) { _spout.ack(tx.getTransactionId()); } @Override public void close() { _spout.close(); } } IBatchSpout _spout; public BatchSpoutExecutor(IBatchSpout spout) { _spout = spout; } @Override public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { return new EmptyCoordinator(); } @Override public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { _spout.open(conf, context); return new BatchSpoutEmitter(); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } @Override public Fields getOutputFields() { return _spout.getOutputFields(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/IBatchID.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; public interface IBatchID { Object getId(); int getAttemptId(); } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/IBatchSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.io.Serializable; import java.util.Map; import storm.trident.operation.TridentCollector; public interface IBatchSpout extends Serializable { void open(Map conf, TopologyContext context); void emitBatch(long batchId, TridentCollector collector); void ack(long batchId); void close(); Map getComponentConfiguration(); Fields getOutputFields(); } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/ICommitterTridentSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import storm.trident.topology.TransactionAttempt; import java.util.Map; public interface ICommitterTridentSpout extends ITridentSpout { public interface Emitter extends ITridentSpout.Emitter { void commit(TransactionAttempt attempt); } @Override public Emitter getEmitter(String txStateId, Map conf, TopologyContext context); } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/IOpaquePartitionedTridentSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.io.Serializable; import java.util.List; import java.util.Map; import storm.trident.operation.TridentCollector; import storm.trident.topology.TransactionAttempt; /** * This defines a transactional spout which does *not* necessarily * replay the same batch every time it emits a batch for a transaction id. * */ public interface IOpaquePartitionedTridentSpout extends Serializable { public interface Coordinator { boolean isReady(long txid); Partitions getPartitionsForBatch(); void close(); } public interface Emitter { /** * Emit a batch of tuples for a partition/transaction. * * Return the metadata describing this batch that will be used as lastPartitionMeta * for defining the parameters of the next batch. */ M emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, Partition partition, M lastPartitionMeta); /** * This method is called when this task is responsible for a new set of partitions. Should be used * to manage things like connections to brokers. */ void refreshPartitions(List partitionResponsibilities); List getOrderedPartitions(Partitions allPartitionInfo); void close(); } Emitter getEmitter(Map conf, TopologyContext context); Coordinator getCoordinator(Map conf, TopologyContext context); Map getComponentConfiguration(); Fields getOutputFields(); } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/IPartitionedTridentSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.io.Serializable; import java.util.List; import java.util.Map; import storm.trident.operation.TridentCollector; import storm.trident.topology.TransactionAttempt; /** * This interface defines a transactional spout that reads its tuples from a partitioned set of * brokers. It automates the storing of metadata for each partition to ensure that the same batch * is always emitted for the same transaction id. The partition metadata is stored in Zookeeper. */ public interface IPartitionedTridentSpout extends Serializable { public interface Coordinator { /** * Return the partitions currently in the source of data. The idea is * is that if a new partition is added and a prior transaction is replayed, it doesn't * emit tuples for the new partition because it knows what partitions were in * that transaction. */ Partitions getPartitionsForBatch(); boolean isReady(long txid); void close(); } public interface Emitter { List getOrderedPartitions(Partitions allPartitionInfo); /** * Emit a batch of tuples for a partition/transaction that's never been emitted before. * Return the metadata that can be used to reconstruct this partition/batch in the future. */ X emitPartitionBatchNew(TransactionAttempt tx, TridentCollector collector, Partition partition, X lastPartitionMeta); /** * This method is called when this task is responsible for a new set of partitions. Should be used * to manage things like connections to brokers. */ void refreshPartitions(List partitionResponsibilities); /** * Emit a batch of tuples for a partition/transaction that has been emitted before, using * the metadata created when it was first emitted. */ void emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, Partition partition, X partitionMeta); void close(); } Coordinator getCoordinator(Map conf, TopologyContext context); Emitter getEmitter(Map conf, TopologyContext context); Map getComponentConfiguration(); Fields getOutputFields(); } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/ISpoutPartition.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; public interface ISpoutPartition { /** * This is used as a Zookeeper node path for storing metadata. */ String getId(); } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/ITridentSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import storm.trident.topology.TransactionAttempt; import backtype.storm.tuple.Fields; import java.io.Serializable; import java.util.Map; import storm.trident.operation.TridentCollector; public interface ITridentSpout extends Serializable { public interface BatchCoordinator { /** * Create metadata for this particular transaction id which has never * been emitted before. The metadata should contain whatever is necessary * to be able to replay the exact batch for the transaction at a later point. * * The metadata is stored in Zookeeper. * * Storm uses the Kryo serializations configured in the component configuration * for this spout to serialize and deserialize the metadata. * * @param txid The id of the transaction. * @param prevMetadata The metadata of the previous transaction * @param currMetadata The metadata for this transaction the last time it was initialized. * null if this is the first attempt * @return the metadata for this new transaction */ X initializeTransaction(long txid, X prevMetadata, X currMetadata); void success(long txid); boolean isReady(long txid); /** * Release any resources from this coordinator. */ void close(); } public interface Emitter { /** * Emit a batch for the specified transaction attempt and metadata for the transaction. The metadata * was created by the Coordinator in the initializeTranaction method. This method must always emit * the same batch of tuples across all tasks for the same transaction id. * */ void emitBatch(TransactionAttempt tx, X coordinatorMeta, TridentCollector collector); /** * This attempt committed successfully, so all state for this commit and before can be safely cleaned up. */ void success(TransactionAttempt tx); /** * Release any resources held by this emitter. */ void close(); } /** * The coordinator for a TransactionalSpout runs in a single thread and indicates when batches * of tuples should be emitted and when transactions should commit. The Coordinator that you provide * in a TransactionalSpout provides metadata for each transaction so that the transactions can be replayed. */ BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context); /** * The emitter for a TransactionalSpout runs as many tasks across the cluster. Emitters are responsible for * emitting batches of tuples for a transaction and must ensure that the same batch of tuples is always * emitted for the same transaction id. */ Emitter getEmitter(String txStateId, Map conf, TopologyContext context); Map getComponentConfiguration(); Fields getOutputFields(); } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; import storm.trident.operation.TridentCollector; import storm.trident.topology.state.RotatingTransactionalState; import storm.trident.topology.state.TransactionalState; import storm.trident.topology.TransactionAttempt; public class OpaquePartitionedTridentSpoutExecutor implements ICommitterTridentSpout { IOpaquePartitionedTridentSpout _spout; public class Coordinator implements ITridentSpout.BatchCoordinator { IOpaquePartitionedTridentSpout.Coordinator _coordinator; public Coordinator(Map conf, TopologyContext context) { _coordinator = _spout.getCoordinator(conf, context); } @Override public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) { return _coordinator.getPartitionsForBatch(); } @Override public void close() { _coordinator.close(); } @Override public void success(long txid) { } @Override public boolean isReady(long txid) { return _coordinator.isReady(txid); } } static class EmitterPartitionState { public RotatingTransactionalState rotatingState; public ISpoutPartition partition; public EmitterPartitionState(RotatingTransactionalState s, ISpoutPartition p) { rotatingState = s; partition = p; } } public class Emitter implements ICommitterTridentSpout.Emitter { IOpaquePartitionedTridentSpout.Emitter _emitter; TransactionalState _state; TreeMap> _cachedMetas = new TreeMap>(); Map _partitionStates = new HashMap(); int _index; int _numTasks; public Emitter(String txStateId, Map conf, TopologyContext context) { _emitter = _spout.getEmitter(conf, context); _index = context.getThisTaskIndex(); _numTasks = context.getComponentTasks(context.getThisComponentId()).size(); _state = TransactionalState.newUserState(conf, txStateId); } Object _savedCoordinatorMeta = null; boolean _changedMeta = false; @Override public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { if(_savedCoordinatorMeta==null || !_savedCoordinatorMeta.equals(coordinatorMeta)) { List partitions = _emitter.getOrderedPartitions(coordinatorMeta); _partitionStates.clear(); List myPartitions = new ArrayList(); for(int i=_index; i < partitions.size(); i+=_numTasks) { ISpoutPartition p = partitions.get(i); String id = p.getId(); myPartitions.add(p); _partitionStates.put(id, new EmitterPartitionState(new RotatingTransactionalState(_state, id), p)); } _emitter.refreshPartitions(myPartitions); _savedCoordinatorMeta = coordinatorMeta; _changedMeta = true; } Map metas = new HashMap(); _cachedMetas.put(tx.getTransactionId(), metas); Entry> entry = _cachedMetas.lowerEntry(tx.getTransactionId()); Map prevCached; if(entry!=null) { prevCached = entry.getValue(); } else { prevCached = new HashMap(); } for(String id: _partitionStates.keySet()) { EmitterPartitionState s = _partitionStates.get(id); s.rotatingState.removeState(tx.getTransactionId()); Object lastMeta = prevCached.get(id); if(lastMeta==null) lastMeta = s.rotatingState.getLastState(); Object meta = _emitter.emitPartitionBatch(tx, collector, s.partition, lastMeta); metas.put(id, meta); } } @Override public void success(TransactionAttempt tx) { for(EmitterPartitionState state: _partitionStates.values()) { state.rotatingState.cleanupBefore(tx.getTransactionId()); } } @Override public void commit(TransactionAttempt attempt) { // this code here handles a case where a previous commit failed, and the partitions // changed since the last commit. This clears out any state for the removed partitions // for this txid. // we make sure only a single task ever does this. we're also guaranteed that // it's impossible for there to be another writer to the directory for that partition // because only a single commit can be happening at once. this is because in order for // another attempt of the batch to commit, the batch phase must have succeeded in between. // hence, all tasks for the prior commit must have finished committing (whether successfully or not) if(_changedMeta && _index==0) { Set validIds = new HashSet(); for(ISpoutPartition p: (List) _emitter.getOrderedPartitions(_savedCoordinatorMeta)) { validIds.add(p.getId()); } for(String existingPartition: _state.list("")) { if(!validIds.contains(existingPartition)) { RotatingTransactionalState s = new RotatingTransactionalState(_state, existingPartition); s.removeState(attempt.getTransactionId()); } } _changedMeta = false; } Long txid = attempt.getTransactionId(); Map metas = _cachedMetas.remove(txid); for(String partitionId: metas.keySet()) { Object meta = metas.get(partitionId); _partitionStates.get(partitionId).rotatingState.overrideState(txid, meta); } } @Override public void close() { _emitter.close(); } } public OpaquePartitionedTridentSpoutExecutor(IOpaquePartitionedTridentSpout spout) { _spout = spout; } @Override public ITridentSpout.BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { return new Coordinator(conf, context); } @Override public ICommitterTridentSpout.Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { return new Emitter(txStateId, conf, context); } @Override public Fields getOutputFields() { return _spout.getOutputFields(); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/PartitionedTridentSpoutExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import storm.trident.operation.TridentCollector; import storm.trident.topology.TransactionAttempt; import storm.trident.topology.state.RotatingTransactionalState; import storm.trident.topology.state.TransactionalState; public class PartitionedTridentSpoutExecutor implements ITridentSpout { IPartitionedTridentSpout _spout; public PartitionedTridentSpoutExecutor(IPartitionedTridentSpout spout) { _spout = spout; } public IPartitionedTridentSpout getPartitionedSpout() { return _spout; } class Coordinator implements ITridentSpout.BatchCoordinator { private IPartitionedTridentSpout.Coordinator _coordinator; public Coordinator(Map conf, TopologyContext context) { _coordinator = _spout.getCoordinator(conf, context); } @Override public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) { if(currMetadata!=null) { return currMetadata; } else { return _coordinator.getPartitionsForBatch(); } } @Override public void close() { _coordinator.close(); } @Override public void success(long txid) { } @Override public boolean isReady(long txid) { return _coordinator.isReady(txid); } } static class EmitterPartitionState { public RotatingTransactionalState rotatingState; public ISpoutPartition partition; public EmitterPartitionState(RotatingTransactionalState s, ISpoutPartition p) { rotatingState = s; partition = p; } } class Emitter implements ITridentSpout.Emitter { private IPartitionedTridentSpout.Emitter _emitter; private TransactionalState _state; private Map _partitionStates = new HashMap(); private int _index; private int _numTasks; public Emitter(String txStateId, Map conf, TopologyContext context) { _emitter = _spout.getEmitter(conf, context); _state = TransactionalState.newUserState(conf, txStateId); _index = context.getThisTaskIndex(); _numTasks = context.getComponentTasks(context.getThisComponentId()).size(); } Object _savedCoordinatorMeta = null; @Override public void emitBatch(final TransactionAttempt tx, final Object coordinatorMeta, final TridentCollector collector) { if(_savedCoordinatorMeta == null || !_savedCoordinatorMeta.equals(coordinatorMeta)) { List partitions = _emitter.getOrderedPartitions(coordinatorMeta); _partitionStates.clear(); List myPartitions = new ArrayList(); for(int i=_index; i < partitions.size(); i+=_numTasks) { ISpoutPartition p = partitions.get(i); String id = p.getId(); myPartitions.add(p); _partitionStates.put(id, new EmitterPartitionState(new RotatingTransactionalState(_state, id), p)); } _emitter.refreshPartitions(myPartitions); _savedCoordinatorMeta = coordinatorMeta; } for(EmitterPartitionState s: _partitionStates.values()) { RotatingTransactionalState state = s.rotatingState; final ISpoutPartition partition = s.partition; Object meta = state.getStateOrCreate(tx.getTransactionId(), new RotatingTransactionalState.StateInitializer() { @Override public Object init(long txid, Object lastState) { return _emitter.emitPartitionBatchNew(tx, collector, partition, lastState); } }); // it's null if one of: // a) a later transaction batch was emitted before this, so we should skip this batch // b) if didn't exist and was created (in which case the StateInitializer was invoked and // it was emitted if(meta!=null) { _emitter.emitPartitionBatch(tx, collector, partition, meta); } } } @Override public void success(TransactionAttempt tx) { for(EmitterPartitionState state: _partitionStates.values()) { state.rotatingState.cleanupBefore(tx.getTransactionId()); } } @Override public void close() { _state.close(); _emitter.close(); } } @Override public ITridentSpout.BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { return new Coordinator(conf, context); } @Override public ITridentSpout.Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { return new Emitter(txStateId, conf, context); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } @Override public Fields getOutputFields() { return _spout.getOutputFields(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/RichSpoutBatchExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.Config; import backtype.storm.spout.ISpoutOutputCollector; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.utils.RotatingMap; import java.util.ArrayList; import java.util.List; import java.util.Map; import storm.trident.operation.TridentCollector; import storm.trident.topology.TransactionAttempt; import storm.trident.util.TridentUtils; public class RichSpoutBatchExecutor implements ITridentSpout { public static final String MAX_BATCH_SIZE_CONF = "topology.spout.max.batch.size"; IRichSpout _spout; public RichSpoutBatchExecutor(IRichSpout spout) { _spout = spout; } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } @Override public Fields getOutputFields() { return TridentUtils.getSingleOutputStreamFields(_spout); } @Override public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { return new RichSpoutCoordinator(); } @Override public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { return new RichSpoutEmitter(conf, context); } class RichSpoutEmitter implements ITridentSpout.Emitter { int _maxBatchSize; boolean prepared = false; CaptureCollector _collector; RotatingMap> idsMap; Map _conf; TopologyContext _context; long lastRotate = System.currentTimeMillis(); long rotateTime; public RichSpoutEmitter(Map conf, TopologyContext context) { _conf = conf; _context = context; Number batchSize = (Number) conf.get(MAX_BATCH_SIZE_CONF); if(batchSize==null) batchSize = 1000; _maxBatchSize = batchSize.intValue(); _collector = new CaptureCollector(); idsMap = new RotatingMap(3); rotateTime = 1000L * ((Number)conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)).intValue(); } @Override public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { long txid = tx.getTransactionId(); long now = System.currentTimeMillis(); if(now - lastRotate > rotateTime) { Map> failed = idsMap.rotate(); for(Long id: failed.keySet()) { //TODO: this isn't right... it's not in the map anymore fail(id); } lastRotate = now; } if(idsMap.containsKey(txid)) { fail(txid); } _collector.reset(collector); if(!prepared) { _spout.open(_conf, _context, new SpoutOutputCollector(_collector)); prepared = true; } for(int i=0; i<_maxBatchSize; i++) { _spout.nextTuple(); if(_collector.numEmitted < i) { break; } } idsMap.put(txid, _collector.ids); } @Override public void success(TransactionAttempt tx) { ack(tx.getTransactionId()); } private void ack(long batchId) { List ids = (List) idsMap.remove(batchId); if(ids!=null) { for(Object id: ids) { _spout.ack(id); } } } private void fail(long batchId) { List ids = (List) idsMap.remove(batchId); if(ids!=null) { for(Object id: ids) { _spout.fail(id); } } } @Override public void close() { } } class RichSpoutCoordinator implements ITridentSpout.BatchCoordinator { @Override public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) { return null; } @Override public void success(long txid) { } @Override public boolean isReady(long txid) { return true; } @Override public void close() { } } static class CaptureCollector implements ISpoutOutputCollector { TridentCollector _collector; public List ids; public int numEmitted; public void reset(TridentCollector c) { _collector = c; ids = new ArrayList(); } @Override public void reportError(Throwable t) { _collector.reportError(t); } @Override public List emit(String stream, List values, Object id) { if(id!=null) ids.add(id); numEmitted++; _collector.emit(values); return null; } @Override public void emitDirect(int task, String stream, List values, Object id) { throw new UnsupportedOperationException("Trident does not support direct streams"); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/RichSpoutBatchId.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; public class RichSpoutBatchId implements IBatchID { long _id; public RichSpoutBatchId(long id) { _id = id; } @Override public Object getId() { // this is to distinguish from TransactionAttempt return this; } @Override public int getAttemptId() { return 0; // each drpc request is always a single attempt } @Override public int hashCode() { return ((Long) _id).hashCode(); } @Override public boolean equals(Object o) { if(!(o instanceof RichSpoutBatchId)) return false; RichSpoutBatchId other = (RichSpoutBatchId) o; return _id == other._id; } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/RichSpoutBatchIdSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; public class RichSpoutBatchIdSerializer extends Serializer { @Override public void write(Kryo kryo, Output output, RichSpoutBatchId id) { output.writeLong(id._id); } @Override public RichSpoutBatchId read(Kryo kryo, Input input, Class type) { long l = input.readLong(); return new RichSpoutBatchId(l); } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/RichSpoutBatchTriggerer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.Config; import backtype.storm.generated.Grouping; import backtype.storm.spout.ISpoutOutputCollector; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichSpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; import java.util.Set; import storm.trident.topology.TridentBoltExecutor; import storm.trident.tuple.ConsList; import storm.trident.util.TridentUtils; public class RichSpoutBatchTriggerer implements IRichSpout { String _stream; IRichSpout _delegate; List _outputTasks; Random _rand; String _coordStream; public RichSpoutBatchTriggerer(IRichSpout delegate, String streamName, String batchGroup) { _delegate = delegate; _stream = streamName; _coordStream = TridentBoltExecutor.COORD_STREAM(batchGroup); } @Override public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _delegate.open(conf, context, new SpoutOutputCollector(new StreamOverrideCollector(collector))); _outputTasks = new ArrayList(); for(String component: Utils.get(context.getThisTargets(), _coordStream, new HashMap()).keySet()) { _outputTasks.addAll(context.getComponentTasks(component)); } _rand = new Random(Utils.secureRandomLong()); } @Override public void close() { _delegate.close(); } @Override public void activate() { _delegate.activate(); } @Override public void deactivate() { _delegate.deactivate(); } @Override public void nextTuple() { _delegate.nextTuple(); } @Override public void ack(Object msgId) { Long batchId = _msgIdToBatchId.remove((Long) msgId); FinishCondition cond = _finishConditions.get(batchId); if(cond!=null) { cond.vals.remove((Long) msgId); if(cond.vals.isEmpty()) { _finishConditions.remove(batchId); _delegate.ack(cond.msgId); } } } @Override public void fail(Object msgId) { Long batchId = _msgIdToBatchId.remove((Long) msgId); FinishCondition cond = _finishConditions.remove(batchId); if(cond!=null) { _delegate.fail(cond.msgId); } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { Fields outFields = TridentUtils.getSingleOutputStreamFields(_delegate); outFields = TridentUtils.fieldsConcat(new Fields("$id$"), outFields); declarer.declareStream(_stream, outFields); // try to find a way to merge this code with what's already done in TridentBoltExecutor declarer.declareStream(_coordStream, true, new Fields("id", "count")); } @Override public Map getComponentConfiguration() { Map conf = _delegate.getComponentConfiguration(); if(conf==null) conf = new HashMap(); else conf = new HashMap(conf); Config.registerSerialization(conf, RichSpoutBatchId.class, RichSpoutBatchIdSerializer.class); return conf; } static class FinishCondition { Set vals = new HashSet(); Object msgId; } Map _msgIdToBatchId = new HashMap(); Map _finishConditions = new HashMap(); class StreamOverrideCollector implements ISpoutOutputCollector { SpoutOutputCollector _collector; public StreamOverrideCollector(SpoutOutputCollector collector) { _collector = collector; } @Override public List emit(String ignore, List values, Object msgId) { long batchIdVal = _rand.nextLong(); Object batchId = new RichSpoutBatchId(batchIdVal); FinishCondition finish = new FinishCondition(); finish.msgId = msgId; List tasks = _collector.emit(_stream, new ConsList(batchId, values)); Set outTasksSet = new HashSet(tasks); for(Integer t: _outputTasks) { int count = 0; if(outTasksSet.contains(t)) { count = 1; } long r = _rand.nextLong(); _collector.emitDirect(t, _coordStream, new Values(batchId, count), r); finish.vals.add(r); } _finishConditions.put(batchIdVal, finish); return tasks; } @Override public void emitDirect(int task, String ignore, List values, Object msgId) { throw new RuntimeException("Trident does not support direct emits from spouts"); } @Override public void reportError(Throwable t) { _collector.reportError(t); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/TridentSpoutCoordinator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.Config; import backtype.storm.task.TopologyContext; import backtype.storm.topology.BasicOutputCollector; import backtype.storm.topology.IBasicBolt; import backtype.storm.topology.OutputFieldsDeclarer; import storm.trident.topology.TransactionAttempt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import storm.trident.topology.MasterBatchCoordinator; import storm.trident.topology.state.RotatingTransactionalState; import storm.trident.topology.state.TransactionalState; public class TridentSpoutCoordinator implements IBasicBolt { public static final Logger LOG = LoggerFactory.getLogger(TridentSpoutCoordinator.class); private static final String META_DIR = "meta"; ITridentSpout _spout; ITridentSpout.BatchCoordinator _coord; RotatingTransactionalState _state; TransactionalState _underlyingState; String _id; public TridentSpoutCoordinator(String id, ITridentSpout spout) { _spout = spout; _id = id; } @Override public void prepare(Map conf, TopologyContext context) { _coord = _spout.getCoordinator(_id, conf, context); _underlyingState = TransactionalState.newCoordinatorState(conf, _id); _state = new RotatingTransactionalState(_underlyingState, META_DIR); } @Override public void execute(Tuple tuple, BasicOutputCollector collector) { TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0); if(tuple.getSourceStreamId().equals(MasterBatchCoordinator.SUCCESS_STREAM_ID)) { _state.cleanupBefore(attempt.getTransactionId()); _coord.success(attempt.getTransactionId()); } else { long txid = attempt.getTransactionId(); Object prevMeta = _state.getPreviousState(txid); Object meta = _coord.initializeTransaction(txid, prevMeta, _state.getState(txid)); _state.overrideState(txid, meta); collector.emit(MasterBatchCoordinator.BATCH_STREAM_ID, new Values(attempt, meta)); } } @Override public void cleanup() { _coord.close(); _underlyingState.close(); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declareStream(MasterBatchCoordinator.BATCH_STREAM_ID, new Fields("tx", "metadata")); } @Override public Map getComponentConfiguration() { Config ret = new Config(); ret.setMaxTaskParallelism(1); return ret; } } ================================================ FILE: storm-core/src/jvm/storm/trident/spout/TridentSpoutExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.spout; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.FailedException; import backtype.storm.topology.OutputFieldsDeclarer; import storm.trident.topology.TransactionAttempt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.TreeMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import storm.trident.operation.TridentCollector; import storm.trident.topology.BatchInfo; import storm.trident.topology.ITridentBatchBolt; import storm.trident.topology.MasterBatchCoordinator; import storm.trident.tuple.ConsList; public class TridentSpoutExecutor implements ITridentBatchBolt { public static String ID_FIELD = "$tx"; public static Logger LOG = LoggerFactory.getLogger(TridentSpoutExecutor.class); AddIdCollector _collector; ITridentSpout _spout; ITridentSpout.Emitter _emitter; String _streamName; String _txStateId; TreeMap _activeBatches = new TreeMap(); public TridentSpoutExecutor(String txStateId, String streamName, ITridentSpout spout) { _txStateId = txStateId; _spout = spout; _streamName = streamName; } @Override public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector) { _emitter = _spout.getEmitter(_txStateId, conf, context); _collector = new AddIdCollector(_streamName, collector); } @Override public void execute(BatchInfo info, Tuple input) { // there won't be a BatchInfo for the success stream TransactionAttempt attempt = (TransactionAttempt) input.getValue(0); if(input.getSourceStreamId().equals(MasterBatchCoordinator.COMMIT_STREAM_ID)) { if(attempt.equals(_activeBatches.get(attempt.getTransactionId()))) { ((ICommitterTridentSpout.Emitter) _emitter).commit(attempt); _activeBatches.remove(attempt.getTransactionId()); } else { throw new FailedException("Received commit for different transaction attempt"); } } else if(input.getSourceStreamId().equals(MasterBatchCoordinator.SUCCESS_STREAM_ID)) { // valid to delete before what's been committed since // those batches will never be accessed again _activeBatches.headMap(attempt.getTransactionId()).clear(); _emitter.success(attempt); } else { _collector.setBatch(info.batchId); _emitter.emitBatch(attempt, input.getValue(1), _collector); _activeBatches.put(attempt.getTransactionId(), attempt); } } @Override public void cleanup() { _emitter.close(); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { List fields = new ArrayList(_spout.getOutputFields().toList()); fields.add(0, ID_FIELD); declarer.declareStream(_streamName, new Fields(fields)); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } @Override public void finishBatch(BatchInfo batchInfo) { } @Override public Object initBatchState(String batchGroup, Object batchId) { return null; } private static class AddIdCollector implements TridentCollector { BatchOutputCollector _delegate; Object _id; String _stream; public AddIdCollector(String stream, BatchOutputCollector c) { _delegate = c; _stream = stream; } public void setBatch(Object id) { _id = id; } @Override public void emit(List values) { _delegate.emit(_stream, new ConsList(_id, values)); } @Override public void reportError(Throwable t) { _delegate.reportError(t); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/BaseQueryFunction.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import storm.trident.operation.BaseOperation; public abstract class BaseQueryFunction extends BaseOperation implements QueryFunction { } ================================================ FILE: storm-core/src/jvm/storm/trident/state/BaseStateUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import storm.trident.operation.BaseOperation; public abstract class BaseStateUpdater extends BaseOperation implements StateUpdater { } ================================================ FILE: storm-core/src/jvm/storm/trident/state/CombinerValueUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import storm.trident.operation.CombinerAggregator; public class CombinerValueUpdater implements ValueUpdater { Object arg; CombinerAggregator agg; public CombinerValueUpdater(CombinerAggregator agg, Object arg) { this.agg = agg; this.arg = arg; } @Override public Object update(Object stored) { if(stored==null) return arg; else return agg.combine(stored, arg); } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/ITupleCollection.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.util.Iterator; import java.util.List; /* Container of a collection of tuples */ public interface ITupleCollection { public Iterator> getTuples(); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/JSONNonTransactionalSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.io.UnsupportedEncodingException; import org.json.simple.JSONValue; public class JSONNonTransactionalSerializer implements Serializer { @Override public byte[] serialize(Object obj) { try { return JSONValue.toJSONString(obj).getBytes("UTF-8"); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } } @Override public Object deserialize(byte[] b) { try { return JSONValue.parse(new String(b, "UTF-8")); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/JSONOpaqueSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.List; import org.json.simple.JSONValue; public class JSONOpaqueSerializer implements Serializer { @Override public byte[] serialize(OpaqueValue obj) { List toSer = new ArrayList(3); toSer.add(obj.currTxid); toSer.add(obj.curr); toSer.add(obj.prev); try { return JSONValue.toJSONString(toSer).getBytes("UTF-8"); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } } @Override public OpaqueValue deserialize(byte[] b) { try { String s = new String(b, "UTF-8"); List deser = (List) JSONValue.parse(s); return new OpaqueValue((Long) deser.get(0), deser.get(1), deser.get(2)); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/JSONTransactionalSerializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.List; import org.json.simple.JSONValue; public class JSONTransactionalSerializer implements Serializer { @Override public byte[] serialize(TransactionalValue obj) { List toSer = new ArrayList(2); toSer.add(obj.getTxid()); toSer.add(obj.getVal()); try { return JSONValue.toJSONString(toSer).getBytes("UTF-8"); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } } @Override public TransactionalValue deserialize(byte[] b) { try { String s = new String(b, "UTF-8"); List deser = (List) JSONValue.parse(s); return new TransactionalValue((Long) deser.get(0), deser.get(1)); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/OpaqueValue.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import org.apache.commons.lang.builder.ToStringBuilder; public class OpaqueValue { Long currTxid; T prev; T curr; public OpaqueValue(Long currTxid, T val, T prev) { this.curr = val; this.currTxid = currTxid; this.prev = prev; } public OpaqueValue(Long currTxid, T val) { this(currTxid, val, null); } public OpaqueValue update(Long batchTxid, T newVal) { T prev; if(batchTxid==null || (this.currTxid < batchTxid)) { prev = this.curr; } else if(batchTxid.equals(this.currTxid)){ prev = this.prev; } else { throw new RuntimeException("Current batch (" + batchTxid + ") is behind state's batch: " + this.toString()); } return new OpaqueValue(batchTxid, newVal, prev); } public T get(Long batchTxid) { if(batchTxid==null || (this.currTxid < batchTxid)) { return curr; } else if(batchTxid.equals(this.currTxid)){ return prev; } else { throw new RuntimeException("Current batch (" + batchTxid + ") is behind state's batch: " + this.toString()); } } public T getCurr() { return curr; } public Long getCurrTxid() { return currTxid; } public T getPrev() { return prev; } @Override public String toString() { return ToStringBuilder.reflectionToString(this); } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/QueryFunction.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.util.List; import storm.trident.operation.EachOperation; import storm.trident.operation.TridentCollector; import storm.trident.tuple.TridentTuple; public interface QueryFunction extends EachOperation { List batchRetrieve(S state, List args); void execute(TridentTuple tuple, T result, TridentCollector collector); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/ReadOnlyState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; public class ReadOnlyState implements State { @Override public void beginCommit(Long txid) { throw new UnsupportedOperationException("This state is read-only and does not support updates"); } @Override public void commit(Long txid) { throw new UnsupportedOperationException("This state is read-only and does not support updates"); } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/ReducerValueUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.util.List; import storm.trident.operation.ReducerAggregator; import storm.trident.tuple.TridentTuple; public class ReducerValueUpdater implements ValueUpdater { List tuples; ReducerAggregator agg; public ReducerValueUpdater(ReducerAggregator agg, List tuples) { this.agg = agg; this.tuples = tuples; } @Override public Object update(Object stored) { Object ret = (stored == null) ? this.agg.init() : stored; for(TridentTuple t: tuples) { ret = this.agg.reduce(ret, t); } return ret; } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/Serializer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.io.Serializable; public interface Serializer extends Serializable { byte[] serialize(T obj); T deserialize(byte[] b); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/State.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; /** * There's 3 different kinds of state: * * 1. non-transactional: ignores commits, updates are permanent. no rollback. a cassandra incrementing state would be like this * 2. repeat-transactional: idempotent as long as all batches for a txid are identical * 3. opaque-transactional: the most general kind of state. updates are always done * based on the previous version of the value if the current commit = latest stored commit * Idempotent even if the batch for a txid can change. * * repeat transactional is idempotent for transactional spouts * opaque transactional is idempotent for opaque or transactional spouts * * Trident should log warnings when state is idempotent but updates will not be idempotent * because of spout */ // retrieving is encapsulated in Retrieval interface public interface State { void beginCommit(Long txid); // can be null for things like partitionPersist occuring off a DRPC stream void commit(Long txid); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/StateFactory.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import backtype.storm.task.IMetricsContext; import java.io.Serializable; import java.util.Map; public interface StateFactory extends Serializable { State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/StateSpec.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.io.Serializable; public class StateSpec implements Serializable { public StateFactory stateFactory; public Integer requiredNumPartitions = null; public StateSpec(StateFactory stateFactory) { this.stateFactory = stateFactory; } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/StateType.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; public enum StateType { NON_TRANSACTIONAL, TRANSACTIONAL, OPAQUE } ================================================ FILE: storm-core/src/jvm/storm/trident/state/StateUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import java.util.List; import storm.trident.operation.Operation; import storm.trident.operation.TridentCollector; import storm.trident.tuple.TridentTuple; public interface StateUpdater extends Operation { // maybe it needs a start phase (where it can do a retrieval, an update phase, and then a finish phase...? // shouldn't really be a one-at-a-time interface, since we have all the tuples already? // TOOD: used for the new values stream // the list is needed to be able to get reduceragg and combineragg persistentaggregate // for grouped streams working efficiently void updateState(S state, List tuples, TridentCollector collector); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/TransactionalValue.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; import org.apache.commons.lang.builder.ToStringBuilder; public class TransactionalValue { T val; Long txid; public TransactionalValue(Long txid, T val) { this.val = val; this.txid = txid; } public T getVal() { return val; } public Long getTxid() { return txid; } @Override public String toString() { return ToStringBuilder.reflectionToString(this); } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/ValueUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state; public interface ValueUpdater { T update(T stored); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/CachedBatchReadsMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import storm.trident.state.ValueUpdater; public class CachedBatchReadsMap { public static class RetVal { public boolean cached; public T val; public RetVal(T v, boolean c) { val = v; cached = c; } } Map, T> _cached = new HashMap, T>(); public IBackingMap _delegate; public CachedBatchReadsMap(IBackingMap delegate) { _delegate = delegate; } public void reset() { _cached.clear(); } public List> multiGet(List> keys) { // TODO: can optimize further by only querying backing map for keys not in the cache List vals = _delegate.multiGet(keys); List> ret = new ArrayList(vals.size()); for(int i=0; i key = keys.get(i); if(_cached.containsKey(key)) { ret.add(new RetVal(_cached.get(key), true)); } else { ret.add(new RetVal(vals.get(i), false)); } } return ret; } public void multiPut(List> keys, List vals) { _delegate.multiPut(keys, vals); cache(keys, vals); } private void cache(List> keys, List vals) { for(int i=0; i key = keys.get(i); T val = vals.get(i); _cached.put(key, val); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/CachedMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import storm.trident.util.LRUMap; /** * Useful to layer over a map that communicates with a database. you generally layer opaque map over this over your database store * @author nathan * @param */ public class CachedMap implements IBackingMap { LRUMap, T> _cache; IBackingMap _delegate; public CachedMap(IBackingMap delegate, int cacheSize) { _cache = new LRUMap, T>(cacheSize); _delegate = delegate; } @Override public List multiGet(List> keys) { Map, T> results = new HashMap, T>(); List> toGet = new ArrayList>(); for(List key: keys) { if(_cache.containsKey(key)) { results.put(key, _cache.get(key)); } else { toGet.add(key); } } List fetchedVals = _delegate.multiGet(toGet); for(int i=0; i key = toGet.get(i); T val = fetchedVals.get(i); _cache.put(key, val); results.put(key, val); } List ret = new ArrayList(keys.size()); for(List key: keys) { ret.add(results.get(key)); } return ret; } @Override public void multiPut(List> keys, List values) { cache(keys, values); _delegate.multiPut(keys, values); } private void cache(List> keys, List values) { for(int i=0; i { List multiGet(List> keys); void multiPut(List> keys, List vals); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/MapCombinerAggStateUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import java.util.ArrayList; import java.util.List; import java.util.Map; import storm.trident.operation.CombinerAggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.state.CombinerValueUpdater; import storm.trident.state.StateUpdater; import storm.trident.state.ValueUpdater; import storm.trident.tuple.ComboList; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class MapCombinerAggStateUpdater implements StateUpdater { CombinerAggregator _agg; Fields _groupFields; Fields _inputFields; ProjectionFactory _groupFactory; ProjectionFactory _inputFactory; ComboList.Factory _factory; public MapCombinerAggStateUpdater(CombinerAggregator agg, Fields groupFields, Fields inputFields) { _agg = agg; _groupFields = groupFields; _inputFields = inputFields; if(inputFields.size()!=1) { throw new IllegalArgumentException("Combiner aggs only take a single field as input. Got this instead: " + inputFields.toString()); } _factory = new ComboList.Factory(groupFields.size(), inputFields.size()); } @Override public void updateState(MapState map, List tuples, TridentCollector collector) { List> groups = new ArrayList>(tuples.size()); List updaters = new ArrayList(tuples.size()); for(TridentTuple t: tuples) { groups.add(_groupFactory.create(t)); updaters.add(new CombinerValueUpdater(_agg,_inputFactory.create(t).getValue(0))); } List newVals = map.multiUpdate(groups, updaters); for(int i=0; i key = groups.get(i); Object result = newVals.get(i); collector.emit(_factory.create(new List[] {key, new Values(result) })); } } @Override public void prepare(Map conf, TridentOperationContext context) { _groupFactory = context.makeProjectionFactory(_groupFields); _inputFactory = context.makeProjectionFactory(_inputFields); } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/MapReducerAggStateUpdater.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import storm.trident.operation.ReducerAggregator; import storm.trident.operation.TridentCollector; import storm.trident.operation.TridentOperationContext; import storm.trident.state.ReducerValueUpdater; import storm.trident.state.StateUpdater; import storm.trident.state.ValueUpdater; import storm.trident.tuple.ComboList; import storm.trident.tuple.TridentTuple; import storm.trident.tuple.TridentTupleView.ProjectionFactory; public class MapReducerAggStateUpdater implements StateUpdater { ReducerAggregator _agg; Fields _groupFields; Fields _inputFields; ProjectionFactory _groupFactory; ProjectionFactory _inputFactory; ComboList.Factory _factory; public MapReducerAggStateUpdater(ReducerAggregator agg, Fields groupFields, Fields inputFields) { _agg = agg; _groupFields = groupFields; _inputFields = inputFields; _factory = new ComboList.Factory(groupFields.size(), 1); } @Override public void updateState(MapState map, List tuples, TridentCollector collector) { Map, List> grouped = new HashMap(); List> groups = new ArrayList>(tuples.size()); List values = new ArrayList(tuples.size()); for(TridentTuple t: tuples) { List group = _groupFactory.create(t); List groupTuples = grouped.get(group); if(groupTuples==null) { groupTuples = new ArrayList(); grouped.put(group, groupTuples); } groupTuples.add(_inputFactory.create(t)); } List> uniqueGroups = new ArrayList(grouped.keySet()); List updaters = new ArrayList(uniqueGroups.size()); for(List group: uniqueGroups) { updaters.add(new ReducerValueUpdater(_agg, grouped.get(group))); } List results = map.multiUpdate(uniqueGroups, updaters); for(int i=0; i group = uniqueGroups.get(i); Object result = results.get(i); collector.emit(_factory.create(new List[] {group, new Values(result) })); } } @Override public void prepare(Map conf, TridentOperationContext context) { _groupFactory = context.makeProjectionFactory(_groupFields); _inputFactory = context.makeProjectionFactory(_inputFields); } @Override public void cleanup() { } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/MapState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import java.util.List; import storm.trident.state.ValueUpdater; public interface MapState extends ReadOnlyMapState { List multiUpdate(List> keys, List updaters); void multiPut(List> keys, List vals); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/MicroBatchIBackingMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import java.io.Serializable; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; public class MicroBatchIBackingMap implements IBackingMap { IBackingMap _delegate; Options _options; public static class Options implements Serializable { public int maxMultiGetBatchSize = 0; // 0 means delegate batch size = trident batch size. public int maxMultiPutBatchSize = 0; } public MicroBatchIBackingMap(final Options options, final IBackingMap delegate) { _options = options; _delegate = delegate; assert options.maxMultiPutBatchSize >= 0; assert options.maxMultiGetBatchSize >= 0; } @Override public void multiPut(final List> keys, final List values) { int thisBatchSize; if(_options.maxMultiPutBatchSize == 0) { thisBatchSize = keys.size(); } else { thisBatchSize = _options.maxMultiPutBatchSize; } LinkedList> keysTodo = new LinkedList>(keys); LinkedList valuesTodo = new LinkedList(values); while(!keysTodo.isEmpty()) { List> keysBatch = new ArrayList>(thisBatchSize); List valuesBatch = new ArrayList(thisBatchSize); for(int i=0; i multiGet(final List> keys) { int thisBatchSize; if(_options.maxMultiGetBatchSize == 0) { thisBatchSize = keys.size(); } else { thisBatchSize = _options.maxMultiGetBatchSize; } LinkedList> keysTodo = new LinkedList>(keys); List ret = new ArrayList(keys.size()); while(!keysTodo.isEmpty()) { List> keysBatch = new ArrayList>(thisBatchSize); for(int i=0; i retSubset = _delegate.multiGet(keysBatch); ret.addAll(retSubset); } return ret; } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/NonTransactionalMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import storm.trident.state.ValueUpdater; import java.util.ArrayList; import java.util.List; public class NonTransactionalMap implements MapState { public static MapState build(IBackingMap backing) { return new NonTransactionalMap(backing); } IBackingMap _backing; protected NonTransactionalMap(IBackingMap backing) { _backing = backing; } @Override public List multiGet(List> keys) { return _backing.multiGet(keys); } @Override public List multiUpdate(List> keys, List updaters) { List curr = _backing.multiGet(keys); List ret = new ArrayList(curr.size()); for(int i=0; i updater = updaters.get(i); ret.add(updater.update(currVal)); } _backing.multiPut(keys, ret); return ret; } @Override public void multiPut(List> keys, List vals) { _backing.multiPut(keys, vals); } @Override public void beginCommit(Long txid) { } @Override public void commit(Long txid) { } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/OpaqueMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import storm.trident.state.OpaqueValue; import storm.trident.state.ValueUpdater; import java.util.ArrayList; import java.util.List; public class OpaqueMap implements MapState { public static MapState build(IBackingMap backing) { return new OpaqueMap(backing); } CachedBatchReadsMap _backing; Long _currTx; protected OpaqueMap(IBackingMap backing) { _backing = new CachedBatchReadsMap(backing); } @Override public List multiGet(List> keys) { List> curr = _backing.multiGet(keys); List ret = new ArrayList(curr.size()); for(CachedBatchReadsMap.RetVal retval: curr) { OpaqueValue val = retval.val; if(val!=null) { ret.add((T) val.get(_currTx)); } else { ret.add(null); } } return ret; } @Override public List multiUpdate(List> keys, List updaters) { List> curr = _backing.multiGet(keys); List newVals = new ArrayList(curr.size()); List ret = new ArrayList(); for(int i=0; i retval = curr.get(i); OpaqueValue val = retval.val; ValueUpdater updater = updaters.get(i); T prev; if(val==null) { prev = null; } else { if(retval.cached) { prev = val.getCurr(); } else { prev = val.get(_currTx); } } T newVal = updater.update(prev); ret.add(newVal); OpaqueValue newOpaqueVal; if(val==null) { newOpaqueVal = new OpaqueValue(_currTx, newVal); } else { newOpaqueVal = val.update(_currTx, newVal); } newVals.add(newOpaqueVal); } _backing.multiPut(keys, newVals); return ret; } @Override public void multiPut(List> keys, List vals) { List updaters = new ArrayList(vals.size()); for(T val: vals) { updaters.add(new ReplaceUpdater(val)); } multiUpdate(keys, updaters); } @Override public void beginCommit(Long txid) { _currTx = txid; _backing.reset(); } @Override public void commit(Long txid) { _currTx = null; _backing.reset(); } static class ReplaceUpdater implements ValueUpdater { T _t; public ReplaceUpdater(T t) { _t = t; } @Override public T update(Object stored) { return _t; } } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/ReadOnlyMapState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import java.util.List; import storm.trident.state.State; public interface ReadOnlyMapState extends State { // certain states might only accept one-tuple keys - those should just throw an error List multiGet(List> keys); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/SnapshottableMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import java.util.Arrays; import java.util.List; import storm.trident.state.ValueUpdater; import storm.trident.state.snapshot.Snapshottable; public class SnapshottableMap implements MapState, Snapshottable { MapState _delegate; List> _keys; public SnapshottableMap(MapState delegate, List snapshotKey) { _delegate = delegate; _keys = Arrays.asList(snapshotKey); } @Override public List multiGet(List> keys) { return _delegate.multiGet(keys); } @Override public List multiUpdate(List> keys, List updaters) { return _delegate.multiUpdate(keys, updaters); } @Override public void multiPut(List> keys, List vals) { _delegate.multiPut(keys, vals); } @Override public void beginCommit(Long txid) { _delegate.beginCommit(txid); } @Override public void commit(Long txid) { _delegate.commit(txid); } @Override public T get() { return multiGet(_keys).get(0); } @Override public T update(ValueUpdater updater) { List updaters = Arrays.asList(updater); return multiUpdate(_keys, updaters).get(0); } @Override public void set(T o) { multiPut(_keys, Arrays.asList(o)); } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/map/TransactionalMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.map; import storm.trident.state.TransactionalValue; import storm.trident.state.ValueUpdater; import java.util.ArrayList; import java.util.List; public class TransactionalMap implements MapState { public static MapState build(IBackingMap backing) { return new TransactionalMap(backing); } CachedBatchReadsMap _backing; Long _currTx; protected TransactionalMap(IBackingMap backing) { _backing = new CachedBatchReadsMap(backing); } @Override public List multiGet(List> keys) { List> vals = _backing.multiGet(keys); List ret = new ArrayList(vals.size()); for(CachedBatchReadsMap.RetVal retval: vals) { TransactionalValue v = retval.val; if(v!=null) { ret.add((T) v.getVal()); } else { ret.add(null); } } return ret; } @Override public List multiUpdate(List> keys, List updaters) { List> curr = _backing.multiGet(keys); List newVals = new ArrayList(curr.size()); List> newKeys = new ArrayList(); List ret = new ArrayList(); for(int i=0; i retval = curr.get(i); TransactionalValue val = retval.val; ValueUpdater updater = updaters.get(i); TransactionalValue newVal; boolean changed = false; if(val==null) { newVal = new TransactionalValue(_currTx, updater.update(null)); changed = true; } else { if(_currTx!=null && _currTx.equals(val.getTxid()) && !retval.cached) { newVal = val; } else { newVal = new TransactionalValue(_currTx, updater.update(val.getVal())); changed = true; } } ret.add(newVal.getVal()); if(changed) { newVals.add(newVal); newKeys.add(keys.get(i)); } } if(!newKeys.isEmpty()) { _backing.multiPut(newKeys, newVals); } return ret; } @Override public void multiPut(List> keys, List vals) { List newVals = new ArrayList(vals.size()); for(T val: vals) { newVals.add(new TransactionalValue(_currTx, val)); } _backing.multiPut(keys, newVals); } @Override public void beginCommit(Long txid) { _currTx = txid; _backing.reset(); } @Override public void commit(Long txid) { _currTx = null; _backing.reset(); } } ================================================ FILE: storm-core/src/jvm/storm/trident/state/snapshot/ReadOnlySnapshottable.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.snapshot; import storm.trident.state.State; public interface ReadOnlySnapshottable extends State { T get(); } ================================================ FILE: storm-core/src/jvm/storm/trident/state/snapshot/Snapshottable.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.state.snapshot; import storm.trident.state.ValueUpdater; // used by Stream#persistentAggregate public interface Snapshottable extends ReadOnlySnapshottable { T update(ValueUpdater updater); void set(T o); } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/CountAsAggregator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import backtype.storm.tuple.Values; import storm.trident.operation.BaseAggregator; import storm.trident.operation.TridentCollector; import storm.trident.tuple.TridentTuple; public class CountAsAggregator extends BaseAggregator { static class State { long count = 0; } @Override public State init(Object batchId, TridentCollector collector) { return new State(); } @Override public void aggregate(State state, TridentTuple tuple, TridentCollector collector) { state.count++; } @Override public void complete(State state, TridentCollector collector) { collector.emit(new Values(state.count)); } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/FeederBatchSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import backtype.storm.utils.RegisteredGlobalState; import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Semaphore; import storm.trident.operation.TridentCollector; import storm.trident.spout.ITridentSpout; import storm.trident.topology.TransactionAttempt; import storm.trident.topology.TridentTopologyBuilder; public class FeederBatchSpout implements ITridentSpout, IFeeder { String _id; String _semaphoreId; Fields _outFields; boolean _waitToEmit = true; public FeederBatchSpout(List fields) { _outFields = new Fields(fields); _id = RegisteredGlobalState.registerState(new CopyOnWriteArrayList()); _semaphoreId = RegisteredGlobalState.registerState(new CopyOnWriteArrayList()); } public void setWaitToEmit(boolean trueIfWait) { _waitToEmit = trueIfWait; } public void feed(Object tuples) { Semaphore sem = new Semaphore(0); ((List)RegisteredGlobalState.getState(_semaphoreId)).add(sem); ((List)RegisteredGlobalState.getState(_id)).add(tuples); try { sem.acquire(); } catch (InterruptedException e) { throw new RuntimeException(e); } } public class FeederCoordinator implements ITridentSpout.BatchCoordinator>>> { int _numPartitions; int _emittedIndex = 0; Map txIndices = new HashMap(); public FeederCoordinator(int numPartitions) { _numPartitions = numPartitions; } @Override public Map>> initializeTransaction(long txid, Map>> prevMetadata, Map>> currMetadata) { if(currMetadata!=null) return currMetadata; List allBatches = (List) RegisteredGlobalState.getState(_id); if(allBatches.size()>_emittedIndex) { Object batchInfo = allBatches.get(_emittedIndex); txIndices.put(txid, _emittedIndex); _emittedIndex += 1; if(batchInfo instanceof Map) { return (Map) batchInfo; } else { List batchList = (List) batchInfo; Map>> partitions = new HashMap(); for(int i=0; i<_numPartitions; i++) { partitions.put(i, new ArrayList()); } for(int i=0; i _masterEmitted) { _masterEmitted++; return true; } else { Utils.sleep(2); return false; } } } public class FeederEmitter implements ITridentSpout.Emitter>>> { int _index; public FeederEmitter(int index) { _index = index; } @Override public void emitBatch(TransactionAttempt tx, Map>> coordinatorMeta, TridentCollector collector) { List> tuples = coordinatorMeta.get(_index); if(tuples!=null) { for(List t: tuples) { collector.emit(t); } } } @Override public void success(TransactionAttempt tx) { } @Override public void close() { } } @Override public Map getComponentConfiguration() { return null; } @Override public Fields getOutputFields() { return _outFields; } @Override public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { int numTasks = context.getComponentTasks( TridentTopologyBuilder.spoutIdFromCoordinatorId( context.getThisComponentId())) .size(); return new FeederCoordinator(numTasks); } @Override public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { return new FeederEmitter(context.getThisTaskIndex()); } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/FeederCommitterBatchSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.List; import java.util.Map; import storm.trident.operation.TridentCollector; import storm.trident.spout.ICommitterTridentSpout; import storm.trident.spout.ITridentSpout; import storm.trident.topology.TransactionAttempt; public class FeederCommitterBatchSpout implements ICommitterTridentSpout, IFeeder { FeederBatchSpout _spout; public FeederCommitterBatchSpout(List fields) { _spout = new FeederBatchSpout(fields); } public void setWaitToEmit(boolean trueIfWait) { _spout.setWaitToEmit(trueIfWait); } static class CommitterEmitter implements ICommitterTridentSpout.Emitter { ITridentSpout.Emitter _emitter; public CommitterEmitter(ITridentSpout.Emitter e) { _emitter = e; } @Override public void commit(TransactionAttempt attempt) { } @Override public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { _emitter.emitBatch(tx, coordinatorMeta, collector); } @Override public void success(TransactionAttempt tx) { _emitter.success(tx); } @Override public void close() { _emitter.close(); } } @Override public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { return new CommitterEmitter(_spout.getEmitter(txStateId, conf, context)); } @Override public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { return _spout.getCoordinator(txStateId, conf, context); } @Override public Fields getOutputFields() { return _spout.getOutputFields(); } @Override public Map getComponentConfiguration() { return _spout.getComponentConfiguration(); } @Override public void feed(Object tuples) { _spout.feed(tuples); } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/FixedBatchSpout.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import backtype.storm.Config; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.HashMap; import storm.trident.operation.TridentCollector; import storm.trident.spout.IBatchSpout; public class FixedBatchSpout implements IBatchSpout { Fields fields; List[] outputs; int maxBatchSize; HashMap>> batches = new HashMap>>(); public FixedBatchSpout(Fields fields, int maxBatchSize, List... outputs) { this.fields = fields; this.outputs = outputs; this.maxBatchSize = maxBatchSize; } int index = 0; boolean cycle = false; public void setCycle(boolean cycle) { this.cycle = cycle; } @Override public void open(Map conf, TopologyContext context) { index = 0; } @Override public void emitBatch(long batchId, TridentCollector collector) { List> batch = this.batches.get(batchId); if(batch == null){ batch = new ArrayList>(); if(index>=outputs.length && cycle) { index = 0; } for(int i=0; index < outputs.length && i < maxBatchSize; index++, i++) { batch.add(outputs[index]); } this.batches.put(batchId, batch); } for(List list : batch){ collector.emit(list); } } @Override public void ack(long batchId) { this.batches.remove(batchId); } @Override public void close() { } @Override public Map getComponentConfiguration() { Config conf = new Config(); conf.setMaxTaskParallelism(1); return conf; } @Override public Fields getOutputFields() { return fields; } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/IFeeder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; public interface IFeeder { void feed(Object tuples); } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/LRUMemoryMapState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import backtype.storm.task.IMetricsContext; import storm.trident.state.ITupleCollection; import backtype.storm.tuple.Values; import java.util.*; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; import storm.trident.state.OpaqueValue; import storm.trident.state.State; import storm.trident.state.StateFactory; import storm.trident.state.ValueUpdater; import storm.trident.state.map.*; import storm.trident.state.snapshot.Snapshottable; import storm.trident.util.LRUMap; public class LRUMemoryMapState implements Snapshottable, ITupleCollection, MapState { LRUMemoryMapStateBacking _backing; SnapshottableMap _delegate; public LRUMemoryMapState(int cacheSize, String id) { _backing = new LRUMemoryMapStateBacking(cacheSize, id); _delegate = new SnapshottableMap(OpaqueMap.build(_backing), new Values("$MEMORY-MAP-STATE-GLOBAL$")); } public T update(ValueUpdater updater) { return _delegate.update(updater); } public void set(T o) { _delegate.set(o); } public T get() { return _delegate.get(); } public void beginCommit(Long txid) { _delegate.beginCommit(txid); } public void commit(Long txid) { _delegate.commit(txid); } public Iterator> getTuples() { return _backing.getTuples(); } public List multiUpdate(List> keys, List updaters) { return _delegate.multiUpdate(keys, updaters); } public void multiPut(List> keys, List vals) { _delegate.multiPut(keys, vals); } public List multiGet(List> keys) { return _delegate.multiGet(keys); } public static class Factory implements StateFactory { String _id; int _maxSize; public Factory(int maxSize) { _id = UUID.randomUUID().toString(); _maxSize = maxSize; } @Override public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) { return new LRUMemoryMapState(_maxSize, _id + partitionIndex); } } static ConcurrentHashMap, Object>> _dbs = new ConcurrentHashMap, Object>>(); static class LRUMemoryMapStateBacking implements IBackingMap, ITupleCollection { public static void clearAll() { _dbs.clear(); } Map, T> db; Long currTx; public LRUMemoryMapStateBacking(int cacheSize, String id) { if (!_dbs.containsKey(id)) { _dbs.put(id, new LRUMap, Object>(cacheSize)); } this.db = (Map, T>) _dbs.get(id); } @Override public List multiGet(List> keys) { List ret = new ArrayList(); for (List key : keys) { ret.add(db.get(key)); } return ret; } @Override public void multiPut(List> keys, List vals) { for (int i = 0; i < keys.size(); i++) { List key = keys.get(i); T val = vals.get(i); db.put(key, val); } } @Override public Iterator> getTuples() { return new Iterator>() { private Iterator, T>> it = db.entrySet().iterator(); public boolean hasNext() { return it.hasNext(); } public List next() { Map.Entry, T> e = it.next(); List ret = new ArrayList(); ret.addAll(e.getKey()); ret.add(((OpaqueValue)e.getValue()).getCurr()); return ret; } public void remove() { throw new UnsupportedOperationException("Not supported yet."); } }; } } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/MemoryBackingMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import storm.trident.state.map.IBackingMap; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; public class MemoryBackingMap implements IBackingMap { Map _vals = new HashMap(); @Override public List multiGet(List> keys) { List ret = new ArrayList(); for(List key: keys) { ret.add(_vals.get(key)); } return ret; } @Override public void multiPut(List> keys, List vals) { for(int i=0; i implements Snapshottable, ITupleCollection, MapState { MemoryMapStateBacking _backing; SnapshottableMap _delegate; public MemoryMapState(String id) { _backing = new MemoryMapStateBacking(id); _delegate = new SnapshottableMap(OpaqueMap.build(_backing), new Values("$MEMORY-MAP-STATE-GLOBAL$")); } public T update(ValueUpdater updater) { return _delegate.update(updater); } public void set(T o) { _delegate.set(o); } public T get() { return _delegate.get(); } public void beginCommit(Long txid) { _delegate.beginCommit(txid); } public void commit(Long txid) { _delegate.commit(txid); } public Iterator> getTuples() { return _backing.getTuples(); } public List multiUpdate(List> keys, List updaters) { return _delegate.multiUpdate(keys, updaters); } public void multiPut(List> keys, List vals) { _delegate.multiPut(keys, vals); } public List multiGet(List> keys) { return _delegate.multiGet(keys); } public static class Factory implements StateFactory { String _id; public Factory() { _id = UUID.randomUUID().toString(); } @Override public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) { return new MemoryMapState(_id + partitionIndex); } } static ConcurrentHashMap, Object>> _dbs = new ConcurrentHashMap, Object>>(); static class MemoryMapStateBacking implements IBackingMap, ITupleCollection { public static void clearAll() { _dbs.clear(); } Map, T> db; Long currTx; public MemoryMapStateBacking(String id) { if (!_dbs.containsKey(id)) { _dbs.put(id, new HashMap()); } this.db = (Map, T>) _dbs.get(id); } @Override public List multiGet(List> keys) { List ret = new ArrayList(); for (List key : keys) { ret.add(db.get(key)); } return ret; } @Override public void multiPut(List> keys, List vals) { for (int i = 0; i < keys.size(); i++) { List key = keys.get(i); T val = vals.get(i); db.put(key, val); } } @Override public Iterator> getTuples() { return new Iterator>() { private Iterator, T>> it = db.entrySet().iterator(); public boolean hasNext() { return it.hasNext(); } public List next() { Map.Entry, T> e = it.next(); List ret = new ArrayList(); ret.addAll(e.getKey()); ret.add(((OpaqueValue)e.getValue()).getCurr()); return ret; } public void remove() { throw new UnsupportedOperationException("Not supported yet."); } }; } } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/MockTridentTuple.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import storm.trident.tuple.TridentTuple; /** * A tuple intended for use in testing. */ public class MockTridentTuple extends ArrayList implements TridentTuple{ private final Map fieldMap; public MockTridentTuple(List fieldNames, List values) { super(values); fieldMap = setupFieldMap(fieldNames); } public MockTridentTuple(List fieldName, Object... values) { super(Arrays.asList(values)); fieldMap = setupFieldMap(fieldName); } private Map setupFieldMap(List fieldNames) { Map newFieldMap = new HashMap(fieldNames.size()); int idx = 0; for (String fieldName : fieldNames) { newFieldMap.put(fieldName, idx++); } return newFieldMap; } private int getIndex(String fieldName) { Integer index = fieldMap.get(fieldName); if (index == null) { throw new IllegalArgumentException("Unknown field name: " + fieldName); } return index; } @Override public List getValues() { return this; } @Override public Object getValue(int i) { return get(i); } @Override public String getString(int i) { return (String)get(i); } @Override public Integer getInteger(int i) { return (Integer)get(i); } @Override public Long getLong(int i) { return (Long)get(i); } @Override public Boolean getBoolean(int i) { return (Boolean)get(i); } @Override public Short getShort(int i) { return (Short)get(i); } @Override public Byte getByte(int i) { return (Byte)get(i); } @Override public Double getDouble(int i) { return (Double)get(i); } @Override public Float getFloat(int i) { return (Float)get(i); } @Override public byte[] getBinary(int i) { return (byte[]) get(i); } @Override public Object getValueByField(String field) { return get(getIndex(field)); } @Override public String getStringByField(String field) { return (String) getValueByField(field); } @Override public Integer getIntegerByField(String field) { return (Integer) getValueByField(field); } @Override public Long getLongByField(String field) { return (Long) getValueByField(field); } @Override public Boolean getBooleanByField(String field) { return (Boolean) getValueByField(field); } @Override public Short getShortByField(String field) { return (Short) getValueByField(field); } @Override public Byte getByteByField(String field) { return (Byte) getValueByField(field); } @Override public Double getDoubleByField(String field) { return (Double) getValueByField(field); } @Override public Float getFloatByField(String field) { return (Float) getValueByField(field); } @Override public byte[] getBinaryByField(String field) { return (byte[]) getValueByField(field); } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/Split.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import backtype.storm.tuple.Values; import storm.trident.operation.BaseFunction; import storm.trident.operation.TridentCollector; import storm.trident.tuple.TridentTuple; public class Split extends BaseFunction { @Override public void execute(TridentTuple tuple, TridentCollector collector) { for(String word: tuple.getString(0).split(" ")) { if(word.length() > 0) { collector.emit(new Values(word)); } } } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/StringLength.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import backtype.storm.tuple.Values; import storm.trident.operation.BaseFunction; import storm.trident.operation.TridentCollector; import storm.trident.tuple.TridentTuple; public class StringLength extends BaseFunction { @Override public void execute(TridentTuple tuple, TridentCollector collector) { collector.emit(new Values(tuple.getString(0).length())); } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/TrueFilter.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import storm.trident.operation.BaseFilter; import storm.trident.tuple.TridentTuple; public class TrueFilter extends BaseFilter { @Override public boolean isKeep(TridentTuple tuple) { return true; } } ================================================ FILE: storm-core/src/jvm/storm/trident/testing/TuplifyArgs.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.testing; import java.util.List; import org.json.simple.JSONValue; import storm.trident.operation.BaseFunction; import storm.trident.operation.TridentCollector; import storm.trident.tuple.TridentTuple; public class TuplifyArgs extends BaseFunction { @Override public void execute(TridentTuple input, TridentCollector collector) { String args = input.getString(0); List> tuples = (List) JSONValue.parse(args); for(List tuple: tuples) { collector.emit(tuple); } } } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/BatchInfo.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology; import storm.trident.spout.IBatchID; public class BatchInfo { public IBatchID batchId; public Object state; public String batchGroup; public BatchInfo(String batchGroup, IBatchID batchId, Object state) { this.batchGroup = batchGroup; this.batchId = batchId; this.state = state; } } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/ITridentBatchBolt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IComponent; import backtype.storm.tuple.Tuple; import java.util.Map; public interface ITridentBatchBolt extends IComponent { void prepare(Map conf, TopologyContext context, BatchOutputCollector collector); void execute(BatchInfo batchInfo, Tuple tuple); void finishBatch(BatchInfo batchInfo); Object initBatchState(String batchGroup, Object batchId); void cleanup(); } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/MasterBatchCoordinator.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology; import backtype.storm.Config; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Values; import backtype.storm.utils.WindowedTimeThrottler; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; import java.util.Random; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import storm.trident.spout.ITridentSpout; import storm.trident.topology.state.TransactionalState; public class MasterBatchCoordinator extends BaseRichSpout { public static final Logger LOG = LoggerFactory.getLogger(MasterBatchCoordinator.class); public static final long INIT_TXID = 1L; public static final String BATCH_STREAM_ID = "$batch"; public static final String COMMIT_STREAM_ID = "$commit"; public static final String SUCCESS_STREAM_ID = "$success"; private static final String CURRENT_TX = "currtx"; private static final String CURRENT_ATTEMPTS = "currattempts"; private List _states = new ArrayList(); TreeMap _activeTx = new TreeMap(); TreeMap _attemptIds; private SpoutOutputCollector _collector; Long _currTransaction; int _maxTransactionActive; List _coordinators = new ArrayList(); List _managedSpoutIds; List _spouts; WindowedTimeThrottler _throttler; boolean _active = true; public MasterBatchCoordinator(List spoutIds, List spouts) { if(spoutIds.isEmpty()) { throw new IllegalArgumentException("Must manage at least one spout"); } _managedSpoutIds = spoutIds; _spouts = spouts; } @Override public void activate() { _active = true; } @Override public void deactivate() { _active = false; } @Override public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _throttler = new WindowedTimeThrottler((Number)conf.get(Config.TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS), 1); for(String spoutId: _managedSpoutIds) { _states.add(TransactionalState.newCoordinatorState(conf, spoutId)); } _currTransaction = getStoredCurrTransaction(); _collector = collector; Number active = (Number) conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); if(active==null) { _maxTransactionActive = 1; } else { _maxTransactionActive = active.intValue(); } _attemptIds = getStoredCurrAttempts(_currTransaction, _maxTransactionActive); for(int i=0; i<_spouts.size(); i++) { String txId = _managedSpoutIds.get(i); _coordinators.add(_spouts.get(i).getCoordinator(txId, conf, context)); } } @Override public void close() { for(TransactionalState state: _states) { state.close(); } } @Override public void nextTuple() { sync(); } @Override public void ack(Object msgId) { TransactionAttempt tx = (TransactionAttempt) msgId; TransactionStatus status = _activeTx.get(tx.getTransactionId()); if(status!=null && tx.equals(status.attempt)) { if(status.status==AttemptStatus.PROCESSING) { status.status = AttemptStatus.PROCESSED; } else if(status.status==AttemptStatus.COMMITTING) { _activeTx.remove(tx.getTransactionId()); _attemptIds.remove(tx.getTransactionId()); _collector.emit(SUCCESS_STREAM_ID, new Values(tx)); _currTransaction = nextTransactionId(tx.getTransactionId()); for(TransactionalState state: _states) { state.setData(CURRENT_TX, _currTransaction); } } sync(); } } @Override public void fail(Object msgId) { TransactionAttempt tx = (TransactionAttempt) msgId; TransactionStatus stored = _activeTx.remove(tx.getTransactionId()); if(stored!=null && tx.equals(stored.attempt)) { _activeTx.tailMap(tx.getTransactionId()).clear(); sync(); } } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { // in partitioned example, in case an emitter task receives a later transaction than it's emitted so far, // when it sees the earlier txid it should know to emit nothing declarer.declareStream(BATCH_STREAM_ID, new Fields("tx")); declarer.declareStream(COMMIT_STREAM_ID, new Fields("tx")); declarer.declareStream(SUCCESS_STREAM_ID, new Fields("tx")); } private void sync() { // note that sometimes the tuples active may be less than max_spout_pending, e.g. // max_spout_pending = 3 // tx 1, 2, 3 active, tx 2 is acked. there won't be a commit for tx 2 (because tx 1 isn't committed yet), // and there won't be a batch for tx 4 because there's max_spout_pending tx active TransactionStatus maybeCommit = _activeTx.get(_currTransaction); if(maybeCommit!=null && maybeCommit.status == AttemptStatus.PROCESSED) { maybeCommit.status = AttemptStatus.COMMITTING; _collector.emit(COMMIT_STREAM_ID, new Values(maybeCommit.attempt), maybeCommit.attempt); } if(_active) { if(_activeTx.size() < _maxTransactionActive) { Long curr = _currTransaction; for(int i=0; i<_maxTransactionActive; i++) { if(!_activeTx.containsKey(curr) && isReady(curr)) { // by using a monotonically increasing attempt id, downstream tasks // can be memory efficient by clearing out state for old attempts // as soon as they see a higher attempt id for a transaction Integer attemptId = _attemptIds.get(curr); if(attemptId==null) { attemptId = 0; } else { attemptId++; } _attemptIds.put(curr, attemptId); for(TransactionalState state: _states) { state.setData(CURRENT_ATTEMPTS, _attemptIds); } TransactionAttempt attempt = new TransactionAttempt(curr, attemptId); _activeTx.put(curr, new TransactionStatus(attempt)); _collector.emit(BATCH_STREAM_ID, new Values(attempt), attempt); _throttler.markEvent(); } curr = nextTransactionId(curr); } } } } private boolean isReady(long txid) { if(_throttler.isThrottled()) return false; //TODO: make this strategy configurable?... right now it goes if anyone is ready for(ITridentSpout.BatchCoordinator coord: _coordinators) { if(coord.isReady(txid)) return true; } return false; } @Override public Map getComponentConfiguration() { Config ret = new Config(); ret.setMaxTaskParallelism(1); ret.registerSerialization(TransactionAttempt.class); return ret; } private static enum AttemptStatus { PROCESSING, PROCESSED, COMMITTING } private static class TransactionStatus { TransactionAttempt attempt; AttemptStatus status; public TransactionStatus(TransactionAttempt attempt) { this.attempt = attempt; this.status = AttemptStatus.PROCESSING; } @Override public String toString() { return attempt.toString() + " <" + status.toString() + ">"; } } private Long nextTransactionId(Long id) { return id + 1; } private Long getStoredCurrTransaction() { Long ret = INIT_TXID; for(TransactionalState state: _states) { Long curr = (Long) state.getData(CURRENT_TX); if(curr!=null && curr.compareTo(ret) > 0) { ret = curr; } } return ret; } private TreeMap getStoredCurrAttempts(long currTransaction, int maxBatches) { TreeMap ret = new TreeMap(); for(TransactionalState state: _states) { Map attempts = (Map) state.getData(CURRENT_ATTEMPTS); if(attempts==null) attempts = new HashMap(); for(Entry e: attempts.entrySet()) { // this is because json doesn't allow numbers as keys... // TODO: replace json with a better form of encoding Number txidObj; if(e.getKey() instanceof String) { txidObj = Long.parseLong((String) e.getKey()); } else { txidObj = (Number) e.getKey(); } long txid = ((Number) txidObj).longValue(); int attemptId = ((Number) e.getValue()).intValue(); Integer curr = ret.get(txid); if(curr==null || attemptId > curr) { ret.put(txid, attemptId); } } } ret.headMap(currTransaction).clear(); ret.tailMap(currTransaction + maxBatches - 1).clear(); return ret; } } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/TransactionAttempt.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology; import storm.trident.spout.IBatchID; public class TransactionAttempt implements IBatchID { Long _txid; int _attemptId; // for kryo compatibility public TransactionAttempt() { } public TransactionAttempt(Long txid, int attemptId) { _txid = txid; _attemptId = attemptId; } public Long getTransactionId() { return _txid; } public Object getId() { return _txid; } public int getAttemptId() { return _attemptId; } @Override public int hashCode() { return _txid.hashCode(); } @Override public boolean equals(Object o) { if(!(o instanceof TransactionAttempt)) return false; TransactionAttempt other = (TransactionAttempt) o; return _txid.equals(other._txid) && _attemptId == other._attemptId; } @Override public String toString() { return "" + _txid + ":" + _attemptId; } } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/TridentBoltExecutor.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology; import backtype.storm.Config; import backtype.storm.Constants; import backtype.storm.coordination.BatchOutputCollector; import backtype.storm.coordination.BatchOutputCollectorImpl; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.task.IOutputCollector; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.FailedException; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.ReportedFailedException; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import backtype.storm.utils.RotatingMap; import backtype.storm.utils.Utils; import java.io.Serializable; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.commons.lang.builder.ToStringBuilder; import storm.trident.spout.IBatchID; public class TridentBoltExecutor implements IRichBolt { public static String COORD_STREAM_PREFIX = "$coord-"; public static String COORD_STREAM(String batch) { return COORD_STREAM_PREFIX + batch; } public static class CoordType implements Serializable { public boolean singleCount; protected CoordType(boolean singleCount) { this.singleCount = singleCount; } public static CoordType single() { return new CoordType(true); } public static CoordType all() { return new CoordType(false); } @Override public boolean equals(Object o) { return singleCount == ((CoordType) o).singleCount; } @Override public String toString() { return ""; } } public static class CoordSpec implements Serializable { public GlobalStreamId commitStream = null; public Map coords = new HashMap(); public CoordSpec() { } } public static class CoordCondition implements Serializable { public GlobalStreamId commitStream; public int expectedTaskReports; Set targetTasks; @Override public String toString() { return ToStringBuilder.reflectionToString(this); } } Map _batchGroupIds; Map _coordSpecs; Map _coordConditions; ITridentBatchBolt _bolt; long _messageTimeoutMs; long _lastRotate; RotatingMap _batches; // map from batchgroupid to coordspec public TridentBoltExecutor(ITridentBatchBolt bolt, Map batchGroupIds, Map coordinationSpecs) { _batchGroupIds = batchGroupIds; _coordSpecs = coordinationSpecs; _bolt = bolt; } public static class TrackedBatch { int attemptId; BatchInfo info; CoordCondition condition; int reportedTasks = 0; int expectedTupleCount = 0; int receivedTuples = 0; Map taskEmittedTuples = new HashMap(); boolean failed = false; boolean receivedCommit; Tuple delayedAck = null; public TrackedBatch(BatchInfo info, CoordCondition condition, int attemptId) { this.info = info; this.condition = condition; this.attemptId = attemptId; receivedCommit = condition.commitStream == null; } @Override public String toString() { return ToStringBuilder.reflectionToString(this); } } public class CoordinatedOutputCollector implements IOutputCollector { IOutputCollector _delegate; TrackedBatch _currBatch = null;; public void setCurrBatch(TrackedBatch batch) { _currBatch = batch; } public CoordinatedOutputCollector(IOutputCollector delegate) { _delegate = delegate; } public List emit(String stream, Collection anchors, List tuple) { List tasks = _delegate.emit(stream, anchors, tuple); updateTaskCounts(tasks); return tasks; } public void emitDirect(int task, String stream, Collection anchors, List tuple) { updateTaskCounts(Arrays.asList(task)); _delegate.emitDirect(task, stream, anchors, tuple); } public void ack(Tuple tuple) { throw new IllegalStateException("Method should never be called"); } public void fail(Tuple tuple) { throw new IllegalStateException("Method should never be called"); } public void reportError(Throwable error) { _delegate.reportError(error); } private void updateTaskCounts(List tasks) { if(_currBatch!=null) { Map taskEmittedTuples = _currBatch.taskEmittedTuples; for(Integer task: tasks) { int newCount = Utils.get(taskEmittedTuples, task, 0) + 1; taskEmittedTuples.put(task, newCount); } } } } OutputCollector _collector; CoordinatedOutputCollector _coordCollector; BatchOutputCollector _coordOutputCollector; TopologyContext _context; @Override public void prepare(Map conf, TopologyContext context, OutputCollector collector) { _messageTimeoutMs = context.maxTopologyMessageTimeout() * 1000L; _lastRotate = System.currentTimeMillis(); _batches = new RotatingMap(2); _context = context; _collector = collector; _coordCollector = new CoordinatedOutputCollector(collector); _coordOutputCollector = new BatchOutputCollectorImpl(new OutputCollector(_coordCollector)); _coordConditions = (Map) context.getExecutorData("__coordConditions"); if(_coordConditions==null) { _coordConditions = new HashMap(); for(String batchGroup: _coordSpecs.keySet()) { CoordSpec spec = _coordSpecs.get(batchGroup); CoordCondition cond = new CoordCondition(); cond.commitStream = spec.commitStream; cond.expectedTaskReports = 0; for(String comp: spec.coords.keySet()) { CoordType ct = spec.coords.get(comp); if(ct.equals(CoordType.single())) { cond.expectedTaskReports+=1; } else { cond.expectedTaskReports+=context.getComponentTasks(comp).size(); } } cond.targetTasks = new HashSet(); for(String component: Utils.get(context.getThisTargets(), COORD_STREAM(batchGroup), new HashMap()).keySet()) { cond.targetTasks.addAll(context.getComponentTasks(component)); } _coordConditions.put(batchGroup, cond); } context.setExecutorData("_coordConditions", _coordConditions); } _bolt.prepare(conf, context, _coordOutputCollector); } private void failBatch(TrackedBatch tracked, FailedException e) { if(e!=null && e instanceof ReportedFailedException) { _collector.reportError(e); } tracked.failed = true; if(tracked.delayedAck!=null) { _collector.fail(tracked.delayedAck); tracked.delayedAck = null; } } private void failBatch(TrackedBatch tracked) { failBatch(tracked, null); } private boolean finishBatch(TrackedBatch tracked, Tuple finishTuple) { boolean success = true; try { _bolt.finishBatch(tracked.info); String stream = COORD_STREAM(tracked.info.batchGroup); for(Integer task: tracked.condition.targetTasks) { _collector.emitDirect(task, stream, finishTuple, new Values(tracked.info.batchId, Utils.get(tracked.taskEmittedTuples, task, 0))); } if(tracked.delayedAck!=null) { _collector.ack(tracked.delayedAck); tracked.delayedAck = null; } } catch(FailedException e) { failBatch(tracked, e); success = false; } _batches.remove(tracked.info.batchId.getId()); return success; } private void checkFinish(TrackedBatch tracked, Tuple tuple, TupleType type) { if(tracked.failed) { failBatch(tracked); _collector.fail(tuple); return; } CoordCondition cond = tracked.condition; boolean delayed = tracked.delayedAck==null && (cond.commitStream!=null && type==TupleType.COMMIT || cond.commitStream==null); if(delayed) { tracked.delayedAck = tuple; } boolean failed = false; if(tracked.receivedCommit && tracked.reportedTasks == cond.expectedTaskReports) { if(tracked.receivedTuples == tracked.expectedTupleCount) { finishBatch(tracked, tuple); } else { //TODO: add logging that not all tuples were received failBatch(tracked); _collector.fail(tuple); failed = true; } } if(!delayed && !failed) { _collector.ack(tuple); } } @Override public void execute(Tuple tuple) { if(tuple.getSourceStreamId().equals(Constants.SYSTEM_TICK_STREAM_ID)) { long now = System.currentTimeMillis(); if(now - _lastRotate > _messageTimeoutMs) { _batches.rotate(); _lastRotate = now; } return; } String batchGroup = _batchGroupIds.get(tuple.getSourceGlobalStreamid()); if(batchGroup==null) { // this is so we can do things like have simple DRPC that doesn't need to use batch processing _coordCollector.setCurrBatch(null); _bolt.execute(null, tuple); _collector.ack(tuple); return; } IBatchID id = (IBatchID) tuple.getValue(0); //get transaction id //if it already exissts and attempt id is greater than the attempt there TrackedBatch tracked = (TrackedBatch) _batches.get(id.getId()); // if(_batches.size() > 10 && _context.getThisTaskIndex() == 0) { // System.out.println("Received in " + _context.getThisComponentId() + " " + _context.getThisTaskIndex() // + " (" + _batches.size() + ")" + // "\ntuple: " + tuple + // "\nwith tracked " + tracked + // "\nwith id " + id + // "\nwith group " + batchGroup // + "\n"); // // } //System.out.println("Num tracked: " + _batches.size() + " " + _context.getThisComponentId() + " " + _context.getThisTaskIndex()); // this code here ensures that only one attempt is ever tracked for a batch, so when // failures happen you don't get an explosion in memory usage in the tasks if(tracked!=null) { if(id.getAttemptId() > tracked.attemptId) { _batches.remove(id.getId()); tracked = null; } else if(id.getAttemptId() < tracked.attemptId) { // no reason to try to execute a previous attempt than we've already seen return; } } if(tracked==null) { tracked = new TrackedBatch(new BatchInfo(batchGroup, id, _bolt.initBatchState(batchGroup, id)), _coordConditions.get(batchGroup), id.getAttemptId()); _batches.put(id.getId(), tracked); } _coordCollector.setCurrBatch(tracked); //System.out.println("TRACKED: " + tracked + " " + tuple); TupleType t = getTupleType(tuple, tracked); if(t==TupleType.COMMIT) { tracked.receivedCommit = true; checkFinish(tracked, tuple, t); } else if(t==TupleType.COORD) { int count = tuple.getInteger(1); tracked.reportedTasks++; tracked.expectedTupleCount+=count; checkFinish(tracked, tuple, t); } else { tracked.receivedTuples++; boolean success = true; try { _bolt.execute(tracked.info, tuple); if(tracked.condition.expectedTaskReports==0) { success = finishBatch(tracked, tuple); } } catch(FailedException e) { failBatch(tracked, e); } if(success) { _collector.ack(tuple); } else { _collector.fail(tuple); } } _coordCollector.setCurrBatch(null); } @Override public void cleanup() { _bolt.cleanup(); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { _bolt.declareOutputFields(declarer); for(String batchGroup: _coordSpecs.keySet()) { declarer.declareStream(COORD_STREAM(batchGroup), true, new Fields("id", "count")); } } @Override public Map getComponentConfiguration() { Map ret = _bolt.getComponentConfiguration(); if(ret==null) ret = new HashMap(); ret.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, 5); // TODO: Need to be able to set the tick tuple time to the message timeout, ideally without parameterization return ret; } private TupleType getTupleType(Tuple tuple, TrackedBatch batch) { CoordCondition cond = batch.condition; if(cond.commitStream!=null && tuple.getSourceGlobalStreamid().equals(cond.commitStream)) { return TupleType.COMMIT; } else if(cond.expectedTaskReports > 0 && tuple.getSourceStreamId().startsWith(COORD_STREAM_PREFIX)) { return TupleType.COORD; } else { return TupleType.REGULAR; } } static enum TupleType { REGULAR, COMMIT, COORD } } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/TridentTopologyBuilder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.generated.StormTopology; import backtype.storm.grouping.CustomStreamGrouping; import backtype.storm.topology.BaseConfigurationDeclarer; import backtype.storm.topology.BoltDeclarer; import backtype.storm.topology.IRichSpout; import backtype.storm.topology.InputDeclarer; import backtype.storm.topology.SpoutDeclarer; import backtype.storm.topology.TopologyBuilder; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.commons.lang.builder.ToStringBuilder; import org.apache.commons.lang.builder.ToStringStyle; import storm.trident.spout.BatchSpoutExecutor; import storm.trident.spout.IBatchSpout; import storm.trident.spout.ICommitterTridentSpout; import storm.trident.spout.ITridentSpout; import storm.trident.spout.RichSpoutBatchTriggerer; import storm.trident.spout.TridentSpoutCoordinator; import storm.trident.spout.TridentSpoutExecutor; import storm.trident.topology.TridentBoltExecutor.CoordSpec; import storm.trident.topology.TridentBoltExecutor.CoordType; // based on transactional topologies public class TridentTopologyBuilder { Map _batchIds = new HashMap(); Map _spouts = new HashMap(); Map _batchPerTupleSpouts = new HashMap(); Map _bolts = new HashMap(); public SpoutDeclarer setBatchPerTupleSpout(String id, String streamName, IRichSpout spout, Integer parallelism, String batchGroup) { Map batchGroups = new HashMap(); batchGroups.put(streamName, batchGroup); markBatchGroups(id, batchGroups); SpoutComponent c = new SpoutComponent(spout, streamName, parallelism, batchGroup); _batchPerTupleSpouts.put(id, c); return new SpoutDeclarerImpl(c); } public SpoutDeclarer setSpout(String id, String streamName, String txStateId, IBatchSpout spout, Integer parallelism, String batchGroup) { return setSpout(id, streamName, txStateId, new BatchSpoutExecutor(spout), parallelism, batchGroup); } public SpoutDeclarer setSpout(String id, String streamName, String txStateId, ITridentSpout spout, Integer parallelism, String batchGroup) { Map batchGroups = new HashMap(); batchGroups.put(streamName, batchGroup); markBatchGroups(id, batchGroups); TransactionalSpoutComponent c = new TransactionalSpoutComponent(spout, streamName, parallelism, txStateId, batchGroup); _spouts.put(id, c); return new SpoutDeclarerImpl(c); } // map from stream name to batch id public BoltDeclarer setBolt(String id, ITridentBatchBolt bolt, Integer parallelism, Set committerBatches, Map batchGroups) { markBatchGroups(id, batchGroups); Component c = new Component(bolt, parallelism, committerBatches); _bolts.put(id, c); return new BoltDeclarerImpl(c); } String masterCoordinator(String batchGroup) { return "$mastercoord-" + batchGroup; } static final String SPOUT_COORD_PREFIX = "$spoutcoord-"; public static String spoutCoordinator(String spoutId) { return SPOUT_COORD_PREFIX + spoutId; } public static String spoutIdFromCoordinatorId(String coordId) { return coordId.substring(SPOUT_COORD_PREFIX.length()); } Map fleshOutStreamBatchIds(boolean includeCommitStream) { Map ret = new HashMap(_batchIds); Set allBatches = new HashSet(_batchIds.values()); for(String b: allBatches) { ret.put(new GlobalStreamId(masterCoordinator(b), MasterBatchCoordinator.BATCH_STREAM_ID), b); if(includeCommitStream) { ret.put(new GlobalStreamId(masterCoordinator(b), MasterBatchCoordinator.COMMIT_STREAM_ID), b); } // DO NOT include the success stream as part of the batch. it should not trigger coordination tuples, // and is just a metadata tuple to assist in cleanup, should not trigger batch tracking } for(String id: _spouts.keySet()) { TransactionalSpoutComponent c = _spouts.get(id); if(c.batchGroupId!=null) { ret.put(new GlobalStreamId(spoutCoordinator(id), MasterBatchCoordinator.BATCH_STREAM_ID), c.batchGroupId); } } //this takes care of setting up coord streams for spouts and bolts for(GlobalStreamId s: _batchIds.keySet()) { String b = _batchIds.get(s); ret.put(new GlobalStreamId(s.get_componentId(), TridentBoltExecutor.COORD_STREAM(b)), b); } return ret; } public StormTopology buildTopology() { TopologyBuilder builder = new TopologyBuilder(); Map batchIdsForSpouts = fleshOutStreamBatchIds(false); Map batchIdsForBolts = fleshOutStreamBatchIds(true); Map> batchesToCommitIds = new HashMap>(); Map> batchesToSpouts = new HashMap>(); for(String id: _spouts.keySet()) { TransactionalSpoutComponent c = _spouts.get(id); if(c.spout instanceof IRichSpout) { //TODO: wrap this to set the stream name builder.setSpout(id, (IRichSpout) c.spout, c.parallelism); } else { String batchGroup = c.batchGroupId; if(!batchesToCommitIds.containsKey(batchGroup)) { batchesToCommitIds.put(batchGroup, new ArrayList()); } batchesToCommitIds.get(batchGroup).add(c.commitStateId); if(!batchesToSpouts.containsKey(batchGroup)) { batchesToSpouts.put(batchGroup, new ArrayList()); } batchesToSpouts.get(batchGroup).add((ITridentSpout) c.spout); BoltDeclarer scd = builder.setBolt(spoutCoordinator(id), new TridentSpoutCoordinator(c.commitStateId, (ITridentSpout) c.spout)) .globalGrouping(masterCoordinator(c.batchGroupId), MasterBatchCoordinator.BATCH_STREAM_ID) .globalGrouping(masterCoordinator(c.batchGroupId), MasterBatchCoordinator.SUCCESS_STREAM_ID); for(Map m: c.componentConfs) { scd.addConfigurations(m); } Map specs = new HashMap(); specs.put(c.batchGroupId, new CoordSpec()); BoltDeclarer bd = builder.setBolt(id, new TridentBoltExecutor( new TridentSpoutExecutor( c.commitStateId, c.streamName, ((ITridentSpout) c.spout)), batchIdsForSpouts, specs), c.parallelism); bd.allGrouping(spoutCoordinator(id), MasterBatchCoordinator.BATCH_STREAM_ID); bd.allGrouping(masterCoordinator(batchGroup), MasterBatchCoordinator.SUCCESS_STREAM_ID); if(c.spout instanceof ICommitterTridentSpout) { bd.allGrouping(masterCoordinator(batchGroup), MasterBatchCoordinator.COMMIT_STREAM_ID); } for(Map m: c.componentConfs) { bd.addConfigurations(m); } } } for(String id: _batchPerTupleSpouts.keySet()) { SpoutComponent c = _batchPerTupleSpouts.get(id); SpoutDeclarer d = builder.setSpout(id, new RichSpoutBatchTriggerer((IRichSpout) c.spout, c.streamName, c.batchGroupId), c.parallelism); for(Map conf: c.componentConfs) { d.addConfigurations(conf); } } for(String batch: batchesToCommitIds.keySet()) { List commitIds = batchesToCommitIds.get(batch); builder.setSpout(masterCoordinator(batch), new MasterBatchCoordinator(commitIds, batchesToSpouts.get(batch))); } for(String id: _bolts.keySet()) { Component c = _bolts.get(id); Map specs = new HashMap(); for(GlobalStreamId s: getBoltSubscriptionStreams(id)) { String batch = batchIdsForBolts.get(s); if(!specs.containsKey(batch)) specs.put(batch, new CoordSpec()); CoordSpec spec = specs.get(batch); CoordType ct; if(_batchPerTupleSpouts.containsKey(s.get_componentId())) { ct = CoordType.single(); } else { ct = CoordType.all(); } spec.coords.put(s.get_componentId(), ct); } for(String b: c.committerBatches) { specs.get(b).commitStream = new GlobalStreamId(masterCoordinator(b), MasterBatchCoordinator.COMMIT_STREAM_ID); } BoltDeclarer d = builder.setBolt(id, new TridentBoltExecutor(c.bolt, batchIdsForBolts, specs), c.parallelism); for(Map conf: c.componentConfs) { d.addConfigurations(conf); } for(InputDeclaration inputDecl: c.declarations) { inputDecl.declare(d); } Map> batchToComponents = getBoltBatchToComponentSubscriptions(id); for(String b: batchToComponents.keySet()) { for(String comp: batchToComponents.get(b)) { d.directGrouping(comp, TridentBoltExecutor.COORD_STREAM(b)); } } for(String b: c.committerBatches) { d.allGrouping(masterCoordinator(b), MasterBatchCoordinator.COMMIT_STREAM_ID); } } return builder.createTopology(); } private void markBatchGroups(String component, Map batchGroups) { for(String stream: batchGroups.keySet()) { _batchIds.put(new GlobalStreamId(component, stream), batchGroups.get(stream)); } } private static class SpoutComponent { public Object spout; public Integer parallelism; public List componentConfs = new ArrayList(); String batchGroupId; String streamName; public SpoutComponent(Object spout, String streamName, Integer parallelism, String batchGroupId) { this.spout = spout; this.streamName = streamName; this.parallelism = parallelism; this.batchGroupId = batchGroupId; } @Override public String toString() { return ToStringBuilder.reflectionToString(this); } } private static class TransactionalSpoutComponent extends SpoutComponent { public String commitStateId; public TransactionalSpoutComponent(Object spout, String streamName, Integer parallelism, String commitStateId, String batchGroupId) { super(spout, streamName, parallelism, batchGroupId); this.commitStateId = commitStateId; } @Override public String toString() { return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE); } } private static class Component { public ITridentBatchBolt bolt; public Integer parallelism; public List declarations = new ArrayList(); public List componentConfs = new ArrayList(); public Set committerBatches; public Component(ITridentBatchBolt bolt, Integer parallelism,Set committerBatches) { this.bolt = bolt; this.parallelism = parallelism; this.committerBatches = committerBatches; } @Override public String toString() { return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE); } } Map> getBoltBatchToComponentSubscriptions(String id) { Map> ret = new HashMap(); for(GlobalStreamId s: getBoltSubscriptionStreams(id)) { String b = _batchIds.get(s); if(!ret.containsKey(b)) ret.put(b, new HashSet()); ret.get(b).add(s.get_componentId()); } return ret; } List getBoltSubscriptionStreams(String id) { List ret = new ArrayList(); Component c = _bolts.get(id); for(InputDeclaration d: c.declarations) { ret.add(new GlobalStreamId(d.getComponent(), d.getStream())); } return ret; } private static interface InputDeclaration { void declare(InputDeclarer declarer); String getComponent(); String getStream(); } private class SpoutDeclarerImpl extends BaseConfigurationDeclarer implements SpoutDeclarer { SpoutComponent _component; public SpoutDeclarerImpl(SpoutComponent component) { _component = component; } @Override public SpoutDeclarer addConfigurations(Map conf) { _component.componentConfs.add(conf); return this; } } private class BoltDeclarerImpl extends BaseConfigurationDeclarer implements BoltDeclarer { Component _component; public BoltDeclarerImpl(Component component) { _component = component; } @Override public BoltDeclarer fieldsGrouping(final String component, final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.fieldsGrouping(component, fields); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer fieldsGrouping(final String component, final String streamId, final Fields fields) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.fieldsGrouping(component, streamId, fields); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer globalGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.globalGrouping(component); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer globalGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.globalGrouping(component, streamId); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer shuffleGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.shuffleGrouping(component); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer shuffleGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.shuffleGrouping(component, streamId); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer localOrShuffleGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.localOrShuffleGrouping(component); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer localOrShuffleGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.localOrShuffleGrouping(component, streamId); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer noneGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.noneGrouping(component); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer noneGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.noneGrouping(component, streamId); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer allGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.allGrouping(component); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer allGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.allGrouping(component, streamId); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer directGrouping(final String component) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.directGrouping(component); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer directGrouping(final String component, final String streamId) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.directGrouping(component, streamId); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer customGrouping(final String component, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.customGrouping(component, grouping); } @Override public String getComponent() { return component; } @Override public String getStream() { return null; } }); return this; } @Override public BoltDeclarer customGrouping(final String component, final String streamId, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.customGrouping(component, streamId, grouping); } @Override public String getComponent() { return component; } @Override public String getStream() { return streamId; } }); return this; } @Override public BoltDeclarer grouping(final GlobalStreamId stream, final Grouping grouping) { addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { declarer.grouping(stream, grouping); } @Override public String getComponent() { return stream.get_componentId(); } @Override public String getStream() { return stream.get_streamId(); } }); return this; } private void addDeclaration(InputDeclaration declaration) { _component.declarations.add(declaration); } @Override public BoltDeclarer addConfigurations(Map conf) { _component.componentConfs.add(conf); return this; } } } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/state/RotatingTransactionalState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology.state; import backtype.storm.utils.Utils; import org.apache.zookeeper.KeeperException; import java.util.HashSet; import java.util.List; import java.util.SortedMap; import java.util.TreeMap; public class RotatingTransactionalState { public static interface StateInitializer { Object init(long txid, Object lastState); } private TransactionalState _state; private String _subdir; private TreeMap _curr = new TreeMap(); public RotatingTransactionalState(TransactionalState state, String subdir) { _state = state; _subdir = subdir; state.mkdir(subdir); sync(); } public Object getLastState() { if(_curr.isEmpty()) return null; else return _curr.lastEntry().getValue(); } public void overrideState(long txid, Object state) { _state.setData(txPath(txid), state); _curr.put(txid, state); } public void removeState(long txid) { if(_curr.containsKey(txid)) { _curr.remove(txid); _state.delete(txPath(txid)); } } public Object getState(long txid) { return _curr.get(txid); } public Object getState(long txid, StateInitializer init) { if(!_curr.containsKey(txid)) { SortedMap prevMap = _curr.headMap(txid); SortedMap afterMap = _curr.tailMap(txid); Long prev = null; if(!prevMap.isEmpty()) prev = prevMap.lastKey(); Object data; if(afterMap.isEmpty()) { Object prevData; if(prev!=null) { prevData = _curr.get(prev); } else { prevData = null; } data = init.init(txid, prevData); } else { data = null; } _curr.put(txid, data); _state.setData(txPath(txid), data); } return _curr.get(txid); } public Object getPreviousState(long txid) { SortedMap prevMap = _curr.headMap(txid); if(prevMap.isEmpty()) return null; else return prevMap.get(prevMap.lastKey()); } public boolean hasCache(long txid) { return _curr.containsKey(txid); } /** * Returns null if it was created, the value otherwise. */ public Object getStateOrCreate(long txid, StateInitializer init) { if(_curr.containsKey(txid)) { return _curr.get(txid); } else { getState(txid, init); return null; } } public void cleanupBefore(long txid) { SortedMap toDelete = _curr.headMap(txid); for(long tx: new HashSet(toDelete.keySet())) { _curr.remove(tx); try { _state.delete(txPath(tx)); } catch(RuntimeException e) { // Ignore NoNodeExists exceptions because when sync() it may populate _curr with stale data since // zookeeper reads are eventually consistent. if(!Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) { throw e; } } } } private void sync() { List txids = _state.list(_subdir); for(String txid_s: txids) { Object data = _state.getData(txPath(txid_s)); _curr.put(Long.parseLong(txid_s), data); } } private String txPath(long tx) { return txPath("" + tx); } private String txPath(String tx) { return _subdir + "/" + tx; } } ================================================ FILE: storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.topology.state; import backtype.storm.Config; import backtype.storm.utils.Utils; import com.netflix.curator.framework.CuratorFramework; import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.json.simple.JSONValue; public class TransactionalState { CuratorFramework _curator; public static TransactionalState newUserState(Map conf, String id) { return new TransactionalState(conf, id, "user"); } public static TransactionalState newCoordinatorState(Map conf, String id) { return new TransactionalState(conf, id, "coordinator"); } protected TransactionalState(Map conf, String id, String subroot) { try { conf = new HashMap(conf); String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) + "/" + id + "/" + subroot; List servers = (List) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS); Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT); CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port); try { initter.create().creatingParentsIfNeeded().forPath(rootDir); } catch(KeeperException.NodeExistsException e) { } initter.close(); _curator = Utils.newCuratorStarted(conf, servers, port, rootDir); } catch (Exception e) { throw new RuntimeException(e); } } public void setData(String path, Object obj) { path = "/" + path; byte[] ser; try { ser = JSONValue.toJSONString(obj).getBytes("UTF-8"); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } try { if(_curator.checkExists().forPath(path)!=null) { _curator.setData().forPath(path, ser); } else { _curator.create() .creatingParentsIfNeeded() .withMode(CreateMode.PERSISTENT) .forPath(path, ser); } } catch(Exception e) { throw new RuntimeException(e); } } public void delete(String path) { path = "/" + path; try { _curator.delete().forPath(path); } catch (Exception e) { throw new RuntimeException(e); } } public List list(String path) { path = "/" + path; try { if(_curator.checkExists().forPath(path)==null) { return new ArrayList(); } else { return _curator.getChildren().forPath(path); } } catch(Exception e) { throw new RuntimeException(e); } } public void mkdir(String path) { setData(path, 7); } public Object getData(String path) { path = "/" + path; try { if(_curator.checkExists().forPath(path)!=null) { return JSONValue.parse(new String(_curator.getData().forPath(path), "UTF-8")); } else { return null; } } catch(Exception e) { throw new RuntimeException(e); } } public void close() { _curator.close(); } private Object getWithBackup(Map amap, Object primary, Object backup) { Object ret = amap.get(primary); if(ret==null) return amap.get(backup); return ret; } } ================================================ FILE: storm-core/src/jvm/storm/trident/tuple/ComboList.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.tuple; import java.io.Serializable; import java.util.AbstractList; import java.util.List; import org.apache.commons.lang.builder.ToStringBuilder; public class ComboList extends AbstractList { public static class Factory implements Serializable { Pointer[] index; int[] sizes; public Factory(int... sizes) { this.sizes = sizes; int total = 0; for(int size: sizes) { total+=size; } index = new Pointer[total]; int i=0; int j=0; for(int size: sizes) { for(int z=0; z { List _elems; Object _first; public ConsList(Object o, List elems) { _elems = elems; _first = o; } @Override public Object get(int i) { if(i==0) return _first; else { return _elems.get(i - 1); } } @Override public int size() { return _elems.size() + 1; } } ================================================ FILE: storm-core/src/jvm/storm/trident/tuple/TridentTuple.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.tuple; import java.io.Serializable; import java.util.List; import java.util.Map; public interface TridentTuple extends List { public static interface Factory extends Serializable { Map getFieldIndex(); List getOutputFields(); int numDelegates(); } List getValues(); Object getValue(int i); String getString(int i); Integer getInteger(int i); Long getLong(int i); Boolean getBoolean(int i); Short getShort(int i); Byte getByte(int i); Double getDouble(int i); Float getFloat(int i); byte[] getBinary(int i); Object getValueByField(String field); String getStringByField(String field); Integer getIntegerByField(String field); Long getLongByField(String field); Boolean getBooleanByField(String field); Short getShortByField(String field); Byte getByteByField(String field); Double getDoubleByField(String field); Float getFloatByField(String field); byte[] getBinaryByField(String field); } ================================================ FILE: storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.tuple; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import clojure.lang.IPersistentVector; import clojure.lang.PersistentVector; import clojure.lang.RT; import java.util.AbstractList; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; //extends abstractlist so that it can be emitted directly as Storm tuples public class TridentTupleView extends AbstractList implements TridentTuple { ValuePointer[] _index; Map _fieldIndex; IPersistentVector _delegates; public static class ProjectionFactory implements Factory { Map _fieldIndex; ValuePointer[] _index; Factory _parent; public ProjectionFactory(Factory parent, Fields projectFields) { _parent = parent; if(projectFields==null) projectFields = new Fields(); Map parentFieldIndex = parent.getFieldIndex(); _fieldIndex = new HashMap(); for(String f: projectFields) { _fieldIndex.put(f, parentFieldIndex.get(f)); } _index = ValuePointer.buildIndex(projectFields, _fieldIndex); } public TridentTuple create(TridentTuple parent) { if(_index.length==0) return EMPTY_TUPLE; else return new TridentTupleView(((TridentTupleView)parent)._delegates, _index, _fieldIndex); } @Override public Map getFieldIndex() { return _fieldIndex; } @Override public int numDelegates() { return _parent.numDelegates(); } @Override public List getOutputFields() { return indexToFieldsList(_index); } } public static class FreshOutputFactory implements Factory { Map _fieldIndex; ValuePointer[] _index; public FreshOutputFactory(Fields selfFields) { _fieldIndex = new HashMap(); for(int i=0; i selfVals) { return new TridentTupleView(PersistentVector.EMPTY.cons(selfVals), _index, _fieldIndex); } @Override public Map getFieldIndex() { return _fieldIndex; } @Override public int numDelegates() { return 1; } @Override public List getOutputFields() { return indexToFieldsList(_index); } } public static class OperationOutputFactory implements Factory { Map _fieldIndex; ValuePointer[] _index; Factory _parent; public OperationOutputFactory(Factory parent, Fields selfFields) { _parent = parent; _fieldIndex = new HashMap(parent.getFieldIndex()); int myIndex = parent.numDelegates(); for(int i=0; i myOrder = new ArrayList(parent.getOutputFields()); Set parentFieldsSet = new HashSet(myOrder); for(String f: selfFields) { if(parentFieldsSet.contains(f)) { throw new IllegalArgumentException( "Additive operations cannot add fields with same name as already exists. " + "Tried adding " + selfFields + " to " + parent.getOutputFields()); } myOrder.add(f); } _index = ValuePointer.buildIndex(new Fields(myOrder), _fieldIndex); } public TridentTuple create(TridentTupleView parent, List selfVals) { IPersistentVector curr = parent._delegates; curr = (IPersistentVector) RT.conj(curr, selfVals); return new TridentTupleView(curr, _index, _fieldIndex); } @Override public Map getFieldIndex() { return _fieldIndex; } @Override public int numDelegates() { return _parent.numDelegates() + 1; } @Override public List getOutputFields() { return indexToFieldsList(_index); } } public static class RootFactory implements Factory { ValuePointer[] index; Map fieldIndex; public RootFactory(Fields inputFields) { index = new ValuePointer[inputFields.size()]; int i=0; for(String f: inputFields) { index[i] = new ValuePointer(0, i, f); i++; } fieldIndex = ValuePointer.buildFieldIndex(index); } public TridentTuple create(Tuple parent) { return new TridentTupleView(PersistentVector.EMPTY.cons(parent.getValues()), index, fieldIndex); } @Override public Map getFieldIndex() { return fieldIndex; } @Override public int numDelegates() { return 1; } @Override public List getOutputFields() { return indexToFieldsList(this.index); } } private static List indexToFieldsList(ValuePointer[] index) { List ret = new ArrayList(); for(ValuePointer p: index) { ret.add(p.field); } return ret; } public static TridentTupleView EMPTY_TUPLE = new TridentTupleView(null, new ValuePointer[0], new HashMap()); // index and fieldIndex are precomputed, delegates built up over many operations using persistent data structures public TridentTupleView(IPersistentVector delegates, ValuePointer[] index, Map fieldIndex) { _delegates = delegates; _index = index; _fieldIndex = fieldIndex; } @Override public List getValues() { return this; } @Override public int size() { return _index.length; } @Override public Object get(int i) { return getValue(i); } @Override public Object getValue(int i) { return getValueByPointer(_index[i]); } @Override public String getString(int i) { return (String) getValue(i); } @Override public Integer getInteger(int i) { return (Integer) getValue(i); } @Override public Long getLong(int i) { return (Long) getValue(i); } @Override public Boolean getBoolean(int i) { return (Boolean) getValue(i); } @Override public Short getShort(int i) { return (Short) getValue(i); } @Override public Byte getByte(int i) { return (Byte) getValue(i); } @Override public Double getDouble(int i) { return (Double) getValue(i); } @Override public Float getFloat(int i) { return (Float) getValue(i); } @Override public byte[] getBinary(int i) { return (byte[]) getValue(i); } @Override public Object getValueByField(String field) { return getValueByPointer(_fieldIndex.get(field)); } @Override public String getStringByField(String field) { return (String) getValueByField(field); } @Override public Integer getIntegerByField(String field) { return (Integer) getValueByField(field); } @Override public Long getLongByField(String field) { return (Long) getValueByField(field); } @Override public Boolean getBooleanByField(String field) { return (Boolean) getValueByField(field); } @Override public Short getShortByField(String field) { return (Short) getValueByField(field); } @Override public Byte getByteByField(String field) { return (Byte) getValueByField(field); } @Override public Double getDoubleByField(String field) { return (Double) getValueByField(field); } @Override public Float getFloatByField(String field) { return (Float) getValueByField(field); } @Override public byte[] getBinaryByField(String field) { return (byte[]) getValueByField(field); } private Object getValueByPointer(ValuePointer ptr) { return ((List)_delegates.nth(ptr.delegateIndex)).get(ptr.index); } } ================================================ FILE: storm-core/src/jvm/storm/trident/tuple/ValuePointer.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.tuple; import backtype.storm.tuple.Fields; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.commons.lang.builder.ToStringBuilder; public class ValuePointer { public static Map buildFieldIndex(ValuePointer[] pointers) { Map ret = new HashMap(); for(ValuePointer ptr: pointers) { ret.put(ptr.field, ptr); } return ret; } public static ValuePointer[] buildIndex(Fields fieldsOrder, Map pointers) { if(fieldsOrder.size()!=pointers.size()) { throw new IllegalArgumentException("Fields order must be same length as pointers map"); } ValuePointer[] ret = new ValuePointer[pointers.size()]; List flist = fieldsOrder.toList(); for(int i=0; i implements Comparable, Serializable { public T source; public T target; public int index; public IndexedEdge(T source, T target, int index) { this.source = source; this.target = target; this.index = index; } @Override public int hashCode() { return 13* source.hashCode() + 7 * target.hashCode() + index; } @Override public boolean equals(Object o) { IndexedEdge other = (IndexedEdge) o; return source.equals(other.source) && target.equals(other.target) && index == other.index; } @Override public int compareTo(Object t) { IndexedEdge other = (IndexedEdge) t; return index - other.index; } } ================================================ FILE: storm-core/src/jvm/storm/trident/util/LRUMap.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.util; import java.util.LinkedHashMap; import java.util.Map; public class LRUMap extends LinkedHashMap { private int _maxSize; public LRUMap(int maxSize) { super(maxSize + 1, 1.0f, true); _maxSize = maxSize; } @Override protected boolean removeEldestEntry(final Map.Entry eldest) { return size() > _maxSize; } } ================================================ FILE: storm-core/src/jvm/storm/trident/util/TridentUtils.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package storm.trident.util; import backtype.storm.generated.StreamInfo; import backtype.storm.topology.IComponent; import backtype.storm.topology.OutputFieldsGetter; import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.logging.Level; import java.util.logging.Logger; import org.apache.thrift7.TBase; import org.apache.thrift7.TDeserializer; import org.apache.thrift7.TException; import org.apache.thrift7.TSerializer; import org.jgrapht.DirectedGraph; public class TridentUtils { public static Fields fieldsUnion(Fields... fields) { Set ret = new HashSet(); for(Fields f: fields) { if(f!=null) ret.addAll(f.toList()); } return new Fields(new ArrayList(ret)); } public static Fields fieldsConcat(Fields... fields) { List ret = new ArrayList(); for(Fields f: fields) { if(f!=null) ret.addAll(f.toList()); } return new Fields(ret); } public static Fields fieldsSubtract(Fields all, Fields minus) { Set removeSet = new HashSet(minus.toList()); List toKeep = new ArrayList(); for(String s: all.toList()) { if(!removeSet.contains(s)) { toKeep.add(s); } } return new Fields(toKeep); } public static Fields getSingleOutputStreamFields(IComponent component) { OutputFieldsGetter getter = new OutputFieldsGetter(); component.declareOutputFields(getter); Map declaration = getter.getFieldsDeclaration(); if(declaration.size()!=1) { throw new RuntimeException("Trident only supports components that emit a single stream"); } StreamInfo si = declaration.values().iterator().next(); if(si.is_direct()) { throw new RuntimeException("Trident does not support direct streams"); } return new Fields(si.get_output_fields()); } /** * Assumes edge contains an index */ public static List getParents(DirectedGraph g, T n) { List incoming = new ArrayList(g.incomingEdgesOf(n)); Collections.sort(incoming); List ret = new ArrayList(); for(IndexedEdge e: incoming) { ret.add((T)e.source); } return ret; } public static List getChildren(DirectedGraph g, T n) { List outgoing = new ArrayList(g.outgoingEdgesOf(n)); List ret = new ArrayList(); for(IndexedEdge e: outgoing) { ret.add((T)e.target); } return ret; } public static T getParent(DirectedGraph g, T n) { List parents = getParents(g, n); if(parents.size()!=1) { throw new RuntimeException("Expected a single parent"); } return parents.get(0); } private static TSerializer ser = new TSerializer(); private static TDeserializer des = new TDeserializer(); public static byte[] thriftSerialize(TBase t) { try { return ser.serialize(t); } catch (TException e) { throw new RuntimeException(e); } } public static T thriftDeserialize(Class c, byte[] b) { try { T ret = (T) c.newInstance(); des.deserialize((TBase) ret, b); return ret; } catch (Exception e) { throw new RuntimeException(e); } } } ================================================ FILE: storm-core/src/multilang/py/storm.py ================================================ # -*- coding: utf-8 -*- # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information # regarding copyright ownership. The ASF licenses this file # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at # # http:# www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. import sys import os import traceback from collections import deque try: import simplejson as json except ImportError: import json json_encode = lambda x: json.dumps(x) json_decode = lambda x: json.loads(x) #reads lines and reconstructs newlines appropriately def readMsg(): msg = "" while True: line = sys.stdin.readline()[0:-1] if line == "end": break msg = msg + line + "\n" return json_decode(msg[0:-1]) MODE = None ANCHOR_TUPLE = None #queue up commands we read while trying to read taskids pending_commands = deque() def readTaskIds(): if pending_taskids: return pending_taskids.popleft() else: msg = readMsg() while type(msg) is not list: pending_commands.append(msg) msg = readMsg() return msg #queue up taskids we read while trying to read commands/tuples pending_taskids = deque() def readCommand(): if pending_commands: return pending_commands.popleft() else: msg = readMsg() while type(msg) is list: pending_taskids.append(msg) msg = readMsg() return msg def readTuple(): cmd = readCommand() return Tuple(cmd["id"], cmd["comp"], cmd["stream"], cmd["task"], cmd["tuple"]) def sendMsgToParent(msg): print json_encode(msg) print "end" sys.stdout.flush() def sync(): sendMsgToParent({'command':'sync'}) def sendpid(heartbeatdir): pid = os.getpid() sendMsgToParent({'pid':pid}) open(heartbeatdir + "/" + str(pid), "w").close() def emit(*args, **kwargs): __emit(*args, **kwargs) return readTaskIds() def emitDirect(task, *args, **kwargs): kwargs["directTask"] = task __emit(*args, **kwargs) def __emit(*args, **kwargs): global MODE if MODE == Bolt: emitBolt(*args, **kwargs) elif MODE == Spout: emitSpout(*args, **kwargs) def emitBolt(tup, stream=None, anchors = [], directTask=None): global ANCHOR_TUPLE if ANCHOR_TUPLE is not None: anchors = [ANCHOR_TUPLE] m = {"command": "emit"} if stream is not None: m["stream"] = stream m["anchors"] = map(lambda a: a.id, anchors) if directTask is not None: m["task"] = directTask m["tuple"] = tup sendMsgToParent(m) def emitSpout(tup, stream=None, id=None, directTask=None): m = {"command": "emit"} if id is not None: m["id"] = id if stream is not None: m["stream"] = stream if directTask is not None: m["task"] = directTask m["tuple"] = tup sendMsgToParent(m) def ack(tup): sendMsgToParent({"command": "ack", "id": tup.id}) def fail(tup): sendMsgToParent({"command": "fail", "id": tup.id}) def reportError(msg): sendMsgToParent({"command": "error", "msg": msg}) def log(msg): sendMsgToParent({"command": "log", "msg": msg}) def initComponent(): setupInfo = readMsg() sendpid(setupInfo['pidDir']) return [setupInfo['conf'], setupInfo['context']] class Tuple(object): def __init__(self, id, component, stream, task, values): self.id = id self.component = component self.stream = stream self.task = task self.values = values def __repr__(self): return '<%s%s>' % ( self.__class__.__name__, ''.join(' %s=%r' % (k, self.__dict__[k]) for k in sorted(self.__dict__.keys()))) class Bolt(object): def initialize(self, stormconf, context): pass def process(self, tuple): pass def run(self): global MODE MODE = Bolt conf, context = initComponent() try: self.initialize(conf, context) while True: tup = readTuple() self.process(tup) except Exception, e: reportError(traceback.format_exc(e)) class BasicBolt(object): def initialize(self, stormconf, context): pass def process(self, tuple): pass def run(self): global MODE MODE = Bolt global ANCHOR_TUPLE conf, context = initComponent() try: self.initialize(conf, context) while True: tup = readTuple() ANCHOR_TUPLE = tup self.process(tup) ack(tup) except Exception, e: reportError(traceback.format_exc(e)) class Spout(object): def initialize(self, conf, context): pass def ack(self, id): pass def fail(self, id): pass def nextTuple(self): pass def run(self): global MODE MODE = Spout conf, context = initComponent() try: self.initialize(conf, context) while True: msg = readCommand() if msg["command"] == "next": self.nextTuple() if msg["command"] == "ack": self.ack(msg["id"]) if msg["command"] == "fail": self.fail(msg["id"]) sync() except Exception, e: reportError(traceback.format_exc(e)) ================================================ FILE: storm-core/src/multilang/rb/storm.rb ================================================ # -*- coding: utf-8 -*- # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information # regarding copyright ownership. The ASF licenses this file # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at # # http:# www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. require "rubygems" require "json" module Storm module Protocol class << self attr_accessor :mode, :pending_taskids, :pending_commands end self.pending_taskids = [] self.pending_commands = [] def read_message msg = "" loop do line = STDIN.readline.chomp break if line == "end" msg << line msg << "\n" end JSON.parse msg.chomp end def read_task_ids Storm::Protocol.pending_taskids.shift || begin msg = read_message until msg.is_a? Array Storm::Protocol.pending_commands.push(msg) msg = read_message end msg end end def read_command Storm::Protocol.pending_commands.shift || begin msg = read_message while msg.is_a? Array Storm::Protocol.pending_taskids.push(msg) msg = read_message end msg end end def send_msg_to_parent(msg) puts msg.to_json puts "end" STDOUT.flush end def sync send_msg_to_parent({'command' => 'sync'}) end def send_pid(heartbeat_dir) pid = Process.pid send_msg_to_parent({'pid' => pid}) File.open("#{heartbeat_dir}/#{pid}", "w").close end def emit_bolt(tup, args = {}) stream = args[:stream] anchors = args[:anchors] || args[:anchor] || [] anchors = [anchors] unless anchors.is_a? Enumerable direct = args[:direct_task] m = {:command => :emit, :anchors => anchors.map(&:id), :tuple => tup} m[:stream] = stream if stream m[:task] = direct if direct send_msg_to_parent m read_task_ids unless direct end def emit_spout(tup, args = {}) stream = args[:stream] id = args[:id] direct = args[:direct_task] m = {:command => :emit, :tuple => tup} m[:id] = id if id m[:stream] = stream if stream m[:task] = direct if direct send_msg_to_parent m read_task_ids unless direct end def emit(*args) case Storm::Protocol.mode when 'spout' emit_spout(*args) when 'bolt' emit_bolt(*args) end end def ack(tup) send_msg_to_parent :command => :ack, :id => tup.id end def fail(tup) send_msg_to_parent :command => :fail, :id => tup.id end def reportError(msg) send_msg_to_parent :command => :error, :msg => msg.to_s end def log(msg) send_msg_to_parent :command => :log, :msg => msg.to_s end def handshake setup_info = read_message send_pid setup_info['pidDir'] [setup_info['conf'], setup_info['context']] end end class Tuple attr_accessor :id, :component, :stream, :task, :values def initialize(id, component, stream, task, values) @id = id @component = component @stream = stream @task = task @values = values end def self.from_hash(hash) Tuple.new(*hash.values_at("id", "comp", "stream", "task", "tuple")) end end class Bolt include Storm::Protocol def prepare(conf, context); end def process(tuple); end def run Storm::Protocol.mode = 'bolt' prepare(*handshake) begin while true process Tuple.from_hash(read_command) end rescue Exception => e reportError 'Exception in bolt: ' + e.message + ' - ' + e.backtrace.join('\n') end end end class Spout include Storm::Protocol def open(conf, context); end def nextTuple; end def ack(id); end def fail(id); end def run Storm::Protocol.mode = 'spout' open(*handshake) begin while true msg = read_command case msg['command'] when 'next' nextTuple when 'ack' ack(msg['id']) when 'fail' fail(msg['id']) end sync end rescue Exception => e reportError 'Exception in spout: ' + e.message + ' - ' + e.backtrace.join('\n') end end end end ================================================ FILE: storm-core/src/py/__init__.py ================================================ ================================================ FILE: storm-core/src/py/storm/DistributedRPC-remote ================================================ #!/usr/bin/env python # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # import sys import pprint from urlparse import urlparse from thrift.transport import TTransport from thrift.transport import TSocket from thrift.transport import THttpClient from thrift.protocol import TBinaryProtocol import DistributedRPC from ttypes import * if len(sys.argv) <= 1 or sys.argv[1] == '--help': print '' print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]' print '' print 'Functions:' print ' string execute(string functionName, string funcArgs)' print '' sys.exit(0) pp = pprint.PrettyPrinter(indent = 2) host = 'localhost' port = 9090 uri = '' framed = False http = False argi = 1 if sys.argv[argi] == '-h': parts = sys.argv[argi+1].split(':') host = parts[0] port = int(parts[1]) argi += 2 if sys.argv[argi] == '-u': url = urlparse(sys.argv[argi+1]) parts = url[1].split(':') host = parts[0] if len(parts) > 1: port = int(parts[1]) else: port = 80 uri = url[2] if url[4]: uri += '?%s' % url[4] http = True argi += 2 if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': framed = True argi += 1 cmd = sys.argv[argi] args = sys.argv[argi+1:] if http: transport = THttpClient.THttpClient(host, port, uri) else: socket = TSocket.TSocket(host, port) if framed: transport = TTransport.TFramedTransport(socket) else: transport = TTransport.TBufferedTransport(socket) protocol = TBinaryProtocol.TBinaryProtocol(transport) client = DistributedRPC.Client(protocol) transport.open() if cmd == 'execute': if len(args) != 2: print 'execute requires 2 args' sys.exit(1) pp.pprint(client.execute(args[0],args[1],)) else: print 'Unrecognized method %s' % cmd sys.exit(1) transport.close() ================================================ FILE: storm-core/src/py/storm/DistributedRPC.py ================================================ # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # from thrift.Thrift import * from ttypes import * from thrift.Thrift import TProcessor from thrift.transport import TTransport from thrift.protocol import TBinaryProtocol, TProtocol try: from thrift.protocol import fastbinary except: fastbinary = None class Iface: def execute(self, functionName, funcArgs): """ Parameters: - functionName - funcArgs """ pass class Client(Iface): def __init__(self, iprot, oprot=None): self._iprot = self._oprot = iprot if oprot is not None: self._oprot = oprot self._seqid = 0 def execute(self, functionName, funcArgs): """ Parameters: - functionName - funcArgs """ self.send_execute(functionName, funcArgs) return self.recv_execute() def send_execute(self, functionName, funcArgs): self._oprot.writeMessageBegin('execute', TMessageType.CALL, self._seqid) args = execute_args() args.functionName = functionName args.funcArgs = funcArgs args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_execute(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = execute_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.e is not None: raise result.e raise TApplicationException(TApplicationException.MISSING_RESULT, "execute failed: unknown result"); class Processor(Iface, TProcessor): def __init__(self, handler): self._handler = handler self._processMap = {} self._processMap["execute"] = Processor.process_execute def process(self, iprot, oprot): (name, type, seqid) = iprot.readMessageBegin() if name not in self._processMap: iprot.skip(TType.STRUCT) iprot.readMessageEnd() x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) x.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() return else: self._processMap[name](self, seqid, iprot, oprot) return True def process_execute(self, seqid, iprot, oprot): args = execute_args() args.read(iprot) iprot.readMessageEnd() result = execute_result() try: result.success = self._handler.execute(args.functionName, args.funcArgs) except DRPCExecutionException, e: result.e = e oprot.writeMessageBegin("execute", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() # HELPER FUNCTIONS AND STRUCTURES class execute_args: """ Attributes: - functionName - funcArgs """ thrift_spec = ( None, # 0 (1, TType.STRING, 'functionName', None, None, ), # 1 (2, TType.STRING, 'funcArgs', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.functionName) + hash(self.funcArgs) def __init__(self, functionName=None, funcArgs=None,): self.functionName = functionName self.funcArgs = funcArgs def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.functionName = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.funcArgs = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('execute_args') if self.functionName is not None: oprot.writeFieldBegin('functionName', TType.STRING, 1) oprot.writeString(self.functionName.encode('utf-8')) oprot.writeFieldEnd() if self.funcArgs is not None: oprot.writeFieldBegin('funcArgs', TType.STRING, 2) oprot.writeString(self.funcArgs.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class execute_result: """ Attributes: - success - e """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 (1, TType.STRUCT, 'e', (DRPCExecutionException, DRPCExecutionException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.success) + hash(self.e) def __init__(self, success=None, e=None,): self.success = success self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRING: self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: self.e = DRPCExecutionException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('execute_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) ================================================ FILE: storm-core/src/py/storm/DistributedRPCInvocations-remote ================================================ #!/usr/bin/env python # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # import sys import pprint from urlparse import urlparse from thrift.transport import TTransport from thrift.transport import TSocket from thrift.transport import THttpClient from thrift.protocol import TBinaryProtocol import DistributedRPCInvocations from ttypes import * if len(sys.argv) <= 1 or sys.argv[1] == '--help': print '' print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]' print '' print 'Functions:' print ' void result(string id, string result)' print ' DRPCRequest fetchRequest(string functionName)' print ' void failRequest(string id)' print '' sys.exit(0) pp = pprint.PrettyPrinter(indent = 2) host = 'localhost' port = 9090 uri = '' framed = False http = False argi = 1 if sys.argv[argi] == '-h': parts = sys.argv[argi+1].split(':') host = parts[0] port = int(parts[1]) argi += 2 if sys.argv[argi] == '-u': url = urlparse(sys.argv[argi+1]) parts = url[1].split(':') host = parts[0] if len(parts) > 1: port = int(parts[1]) else: port = 80 uri = url[2] if url[4]: uri += '?%s' % url[4] http = True argi += 2 if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': framed = True argi += 1 cmd = sys.argv[argi] args = sys.argv[argi+1:] if http: transport = THttpClient.THttpClient(host, port, uri) else: socket = TSocket.TSocket(host, port) if framed: transport = TTransport.TFramedTransport(socket) else: transport = TTransport.TBufferedTransport(socket) protocol = TBinaryProtocol.TBinaryProtocol(transport) client = DistributedRPCInvocations.Client(protocol) transport.open() if cmd == 'result': if len(args) != 2: print 'result requires 2 args' sys.exit(1) pp.pprint(client.result(args[0],args[1],)) elif cmd == 'fetchRequest': if len(args) != 1: print 'fetchRequest requires 1 args' sys.exit(1) pp.pprint(client.fetchRequest(args[0],)) elif cmd == 'failRequest': if len(args) != 1: print 'failRequest requires 1 args' sys.exit(1) pp.pprint(client.failRequest(args[0],)) else: print 'Unrecognized method %s' % cmd sys.exit(1) transport.close() ================================================ FILE: storm-core/src/py/storm/DistributedRPCInvocations.py ================================================ # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # from thrift.Thrift import * from ttypes import * from thrift.Thrift import TProcessor from thrift.transport import TTransport from thrift.protocol import TBinaryProtocol, TProtocol try: from thrift.protocol import fastbinary except: fastbinary = None class Iface: def result(self, id, result): """ Parameters: - id - result """ pass def fetchRequest(self, functionName): """ Parameters: - functionName """ pass def failRequest(self, id): """ Parameters: - id """ pass class Client(Iface): def __init__(self, iprot, oprot=None): self._iprot = self._oprot = iprot if oprot is not None: self._oprot = oprot self._seqid = 0 def result(self, id, result): """ Parameters: - id - result """ self.send_result(id, result) self.recv_result() def send_result(self, id, result): self._oprot.writeMessageBegin('result', TMessageType.CALL, self._seqid) args = result_args() args.id = id args.result = result args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_result(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = result_result() result.read(self._iprot) self._iprot.readMessageEnd() return def fetchRequest(self, functionName): """ Parameters: - functionName """ self.send_fetchRequest(functionName) return self.recv_fetchRequest() def send_fetchRequest(self, functionName): self._oprot.writeMessageBegin('fetchRequest', TMessageType.CALL, self._seqid) args = fetchRequest_args() args.functionName = functionName args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_fetchRequest(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = fetchRequest_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result"); def failRequest(self, id): """ Parameters: - id """ self.send_failRequest(id) self.recv_failRequest() def send_failRequest(self, id): self._oprot.writeMessageBegin('failRequest', TMessageType.CALL, self._seqid) args = failRequest_args() args.id = id args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_failRequest(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = failRequest_result() result.read(self._iprot) self._iprot.readMessageEnd() return class Processor(Iface, TProcessor): def __init__(self, handler): self._handler = handler self._processMap = {} self._processMap["result"] = Processor.process_result self._processMap["fetchRequest"] = Processor.process_fetchRequest self._processMap["failRequest"] = Processor.process_failRequest def process(self, iprot, oprot): (name, type, seqid) = iprot.readMessageBegin() if name not in self._processMap: iprot.skip(TType.STRUCT) iprot.readMessageEnd() x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) x.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() return else: self._processMap[name](self, seqid, iprot, oprot) return True def process_result(self, seqid, iprot, oprot): args = result_args() args.read(iprot) iprot.readMessageEnd() result = result_result() self._handler.result(args.id, args.result) oprot.writeMessageBegin("result", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_fetchRequest(self, seqid, iprot, oprot): args = fetchRequest_args() args.read(iprot) iprot.readMessageEnd() result = fetchRequest_result() result.success = self._handler.fetchRequest(args.functionName) oprot.writeMessageBegin("fetchRequest", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_failRequest(self, seqid, iprot, oprot): args = failRequest_args() args.read(iprot) iprot.readMessageEnd() result = failRequest_result() self._handler.failRequest(args.id) oprot.writeMessageBegin("failRequest", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() # HELPER FUNCTIONS AND STRUCTURES class result_args: """ Attributes: - id - result """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 (2, TType.STRING, 'result', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.id) + hash(self.result) def __init__(self, id=None, result=None,): self.id = id self.result = result def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.result = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('result_args') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() if self.result is not None: oprot.writeFieldBegin('result', TType.STRING, 2) oprot.writeString(self.result.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class result_result: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('result_result') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class fetchRequest_args: """ Attributes: - functionName """ thrift_spec = ( None, # 0 (1, TType.STRING, 'functionName', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.functionName) def __init__(self, functionName=None,): self.functionName = functionName def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.functionName = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('fetchRequest_args') if self.functionName is not None: oprot.writeFieldBegin('functionName', TType.STRING, 1) oprot.writeString(self.functionName.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class fetchRequest_result: """ Attributes: - success """ thrift_spec = ( (0, TType.STRUCT, 'success', (DRPCRequest, DRPCRequest.thrift_spec), None, ), # 0 ) def __hash__(self): return 0 + hash(self.success) def __init__(self, success=None,): self.success = success def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRUCT: self.success = DRPCRequest() self.success.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('fetchRequest_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class failRequest_args: """ Attributes: - id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.id) def __init__(self, id=None,): self.id = id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('failRequest_args') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class failRequest_result: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('failRequest_result') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) ================================================ FILE: storm-core/src/py/storm/Nimbus-remote ================================================ #!/usr/bin/env python # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # import sys import pprint from urlparse import urlparse from thrift.transport import TTransport from thrift.transport import TSocket from thrift.transport import THttpClient from thrift.protocol import TBinaryProtocol import Nimbus from ttypes import * if len(sys.argv) <= 1 or sys.argv[1] == '--help': print '' print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]' print '' print 'Functions:' print ' void submitTopology(string name, string uploadedJarLocation, string jsonConf, StormTopology topology)' print ' void submitTopologyWithOpts(string name, string uploadedJarLocation, string jsonConf, StormTopology topology, SubmitOptions options)' print ' void killTopology(string name)' print ' void killTopologyWithOpts(string name, KillOptions options)' print ' void activate(string name)' print ' void deactivate(string name)' print ' void rebalance(string name, RebalanceOptions options)' print ' string beginFileUpload()' print ' void uploadChunk(string location, string chunk)' print ' void finishFileUpload(string location)' print ' string beginFileDownload(string file)' print ' string downloadChunk(string id)' print ' string getNimbusConf()' print ' ClusterSummary getClusterInfo()' print ' TopologyInfo getTopologyInfo(string id)' print ' string getTopologyConf(string id)' print ' StormTopology getTopology(string id)' print ' StormTopology getUserTopology(string id)' print '' sys.exit(0) pp = pprint.PrettyPrinter(indent = 2) host = 'localhost' port = 9090 uri = '' framed = False http = False argi = 1 if sys.argv[argi] == '-h': parts = sys.argv[argi+1].split(':') host = parts[0] port = int(parts[1]) argi += 2 if sys.argv[argi] == '-u': url = urlparse(sys.argv[argi+1]) parts = url[1].split(':') host = parts[0] if len(parts) > 1: port = int(parts[1]) else: port = 80 uri = url[2] if url[4]: uri += '?%s' % url[4] http = True argi += 2 if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': framed = True argi += 1 cmd = sys.argv[argi] args = sys.argv[argi+1:] if http: transport = THttpClient.THttpClient(host, port, uri) else: socket = TSocket.TSocket(host, port) if framed: transport = TTransport.TFramedTransport(socket) else: transport = TTransport.TBufferedTransport(socket) protocol = TBinaryProtocol.TBinaryProtocol(transport) client = Nimbus.Client(protocol) transport.open() if cmd == 'submitTopology': if len(args) != 4: print 'submitTopology requires 4 args' sys.exit(1) pp.pprint(client.submitTopology(args[0],args[1],args[2],eval(args[3]),)) elif cmd == 'submitTopologyWithOpts': if len(args) != 5: print 'submitTopologyWithOpts requires 5 args' sys.exit(1) pp.pprint(client.submitTopologyWithOpts(args[0],args[1],args[2],eval(args[3]),eval(args[4]),)) elif cmd == 'killTopology': if len(args) != 1: print 'killTopology requires 1 args' sys.exit(1) pp.pprint(client.killTopology(args[0],)) elif cmd == 'killTopologyWithOpts': if len(args) != 2: print 'killTopologyWithOpts requires 2 args' sys.exit(1) pp.pprint(client.killTopologyWithOpts(args[0],eval(args[1]),)) elif cmd == 'activate': if len(args) != 1: print 'activate requires 1 args' sys.exit(1) pp.pprint(client.activate(args[0],)) elif cmd == 'deactivate': if len(args) != 1: print 'deactivate requires 1 args' sys.exit(1) pp.pprint(client.deactivate(args[0],)) elif cmd == 'rebalance': if len(args) != 2: print 'rebalance requires 2 args' sys.exit(1) pp.pprint(client.rebalance(args[0],eval(args[1]),)) elif cmd == 'beginFileUpload': if len(args) != 0: print 'beginFileUpload requires 0 args' sys.exit(1) pp.pprint(client.beginFileUpload()) elif cmd == 'uploadChunk': if len(args) != 2: print 'uploadChunk requires 2 args' sys.exit(1) pp.pprint(client.uploadChunk(args[0],args[1],)) elif cmd == 'finishFileUpload': if len(args) != 1: print 'finishFileUpload requires 1 args' sys.exit(1) pp.pprint(client.finishFileUpload(args[0],)) elif cmd == 'beginFileDownload': if len(args) != 1: print 'beginFileDownload requires 1 args' sys.exit(1) pp.pprint(client.beginFileDownload(args[0],)) elif cmd == 'downloadChunk': if len(args) != 1: print 'downloadChunk requires 1 args' sys.exit(1) pp.pprint(client.downloadChunk(args[0],)) elif cmd == 'getNimbusConf': if len(args) != 0: print 'getNimbusConf requires 0 args' sys.exit(1) pp.pprint(client.getNimbusConf()) elif cmd == 'getClusterInfo': if len(args) != 0: print 'getClusterInfo requires 0 args' sys.exit(1) pp.pprint(client.getClusterInfo()) elif cmd == 'getTopologyInfo': if len(args) != 1: print 'getTopologyInfo requires 1 args' sys.exit(1) pp.pprint(client.getTopologyInfo(args[0],)) elif cmd == 'getTopologyConf': if len(args) != 1: print 'getTopologyConf requires 1 args' sys.exit(1) pp.pprint(client.getTopologyConf(args[0],)) elif cmd == 'getTopology': if len(args) != 1: print 'getTopology requires 1 args' sys.exit(1) pp.pprint(client.getTopology(args[0],)) elif cmd == 'getUserTopology': if len(args) != 1: print 'getUserTopology requires 1 args' sys.exit(1) pp.pprint(client.getUserTopology(args[0],)) else: print 'Unrecognized method %s' % cmd sys.exit(1) transport.close() ================================================ FILE: storm-core/src/py/storm/Nimbus.py ================================================ # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # from thrift.Thrift import * from ttypes import * from thrift.Thrift import TProcessor from thrift.transport import TTransport from thrift.protocol import TBinaryProtocol, TProtocol try: from thrift.protocol import fastbinary except: fastbinary = None class Iface: def submitTopology(self, name, uploadedJarLocation, jsonConf, topology): """ Parameters: - name - uploadedJarLocation - jsonConf - topology """ pass def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): """ Parameters: - name - uploadedJarLocation - jsonConf - topology - options """ pass def killTopology(self, name): """ Parameters: - name """ pass def killTopologyWithOpts(self, name, options): """ Parameters: - name - options """ pass def activate(self, name): """ Parameters: - name """ pass def deactivate(self, name): """ Parameters: - name """ pass def rebalance(self, name, options): """ Parameters: - name - options """ pass def beginFileUpload(self, ): pass def uploadChunk(self, location, chunk): """ Parameters: - location - chunk """ pass def finishFileUpload(self, location): """ Parameters: - location """ pass def beginFileDownload(self, file): """ Parameters: - file """ pass def downloadChunk(self, id): """ Parameters: - id """ pass def getNimbusConf(self, ): pass def getClusterInfo(self, ): pass def getTopologyInfo(self, id): """ Parameters: - id """ pass def getTopologyConf(self, id): """ Parameters: - id """ pass def getTopology(self, id): """ Parameters: - id """ pass def getUserTopology(self, id): """ Parameters: - id """ pass class Client(Iface): def __init__(self, iprot, oprot=None): self._iprot = self._oprot = iprot if oprot is not None: self._oprot = oprot self._seqid = 0 def submitTopology(self, name, uploadedJarLocation, jsonConf, topology): """ Parameters: - name - uploadedJarLocation - jsonConf - topology """ self.send_submitTopology(name, uploadedJarLocation, jsonConf, topology) self.recv_submitTopology() def send_submitTopology(self, name, uploadedJarLocation, jsonConf, topology): self._oprot.writeMessageBegin('submitTopology', TMessageType.CALL, self._seqid) args = submitTopology_args() args.name = name args.uploadedJarLocation = uploadedJarLocation args.jsonConf = jsonConf args.topology = topology args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_submitTopology(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = submitTopology_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.e is not None: raise result.e if result.ite is not None: raise result.ite return def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): """ Parameters: - name - uploadedJarLocation - jsonConf - topology - options """ self.send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options) self.recv_submitTopologyWithOpts() def send_submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): self._oprot.writeMessageBegin('submitTopologyWithOpts', TMessageType.CALL, self._seqid) args = submitTopologyWithOpts_args() args.name = name args.uploadedJarLocation = uploadedJarLocation args.jsonConf = jsonConf args.topology = topology args.options = options args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_submitTopologyWithOpts(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = submitTopologyWithOpts_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.e is not None: raise result.e if result.ite is not None: raise result.ite return def killTopology(self, name): """ Parameters: - name """ self.send_killTopology(name) self.recv_killTopology() def send_killTopology(self, name): self._oprot.writeMessageBegin('killTopology', TMessageType.CALL, self._seqid) args = killTopology_args() args.name = name args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_killTopology(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = killTopology_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.e is not None: raise result.e return def killTopologyWithOpts(self, name, options): """ Parameters: - name - options """ self.send_killTopologyWithOpts(name, options) self.recv_killTopologyWithOpts() def send_killTopologyWithOpts(self, name, options): self._oprot.writeMessageBegin('killTopologyWithOpts', TMessageType.CALL, self._seqid) args = killTopologyWithOpts_args() args.name = name args.options = options args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_killTopologyWithOpts(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = killTopologyWithOpts_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.e is not None: raise result.e return def activate(self, name): """ Parameters: - name """ self.send_activate(name) self.recv_activate() def send_activate(self, name): self._oprot.writeMessageBegin('activate', TMessageType.CALL, self._seqid) args = activate_args() args.name = name args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_activate(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = activate_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.e is not None: raise result.e return def deactivate(self, name): """ Parameters: - name """ self.send_deactivate(name) self.recv_deactivate() def send_deactivate(self, name): self._oprot.writeMessageBegin('deactivate', TMessageType.CALL, self._seqid) args = deactivate_args() args.name = name args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_deactivate(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = deactivate_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.e is not None: raise result.e return def rebalance(self, name, options): """ Parameters: - name - options """ self.send_rebalance(name, options) self.recv_rebalance() def send_rebalance(self, name, options): self._oprot.writeMessageBegin('rebalance', TMessageType.CALL, self._seqid) args = rebalance_args() args.name = name args.options = options args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_rebalance(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = rebalance_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.e is not None: raise result.e if result.ite is not None: raise result.ite return def beginFileUpload(self, ): self.send_beginFileUpload() return self.recv_beginFileUpload() def send_beginFileUpload(self, ): self._oprot.writeMessageBegin('beginFileUpload', TMessageType.CALL, self._seqid) args = beginFileUpload_args() args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_beginFileUpload(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = beginFileUpload_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); def uploadChunk(self, location, chunk): """ Parameters: - location - chunk """ self.send_uploadChunk(location, chunk) self.recv_uploadChunk() def send_uploadChunk(self, location, chunk): self._oprot.writeMessageBegin('uploadChunk', TMessageType.CALL, self._seqid) args = uploadChunk_args() args.location = location args.chunk = chunk args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_uploadChunk(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = uploadChunk_result() result.read(self._iprot) self._iprot.readMessageEnd() return def finishFileUpload(self, location): """ Parameters: - location """ self.send_finishFileUpload(location) self.recv_finishFileUpload() def send_finishFileUpload(self, location): self._oprot.writeMessageBegin('finishFileUpload', TMessageType.CALL, self._seqid) args = finishFileUpload_args() args.location = location args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_finishFileUpload(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = finishFileUpload_result() result.read(self._iprot) self._iprot.readMessageEnd() return def beginFileDownload(self, file): """ Parameters: - file """ self.send_beginFileDownload(file) return self.recv_beginFileDownload() def send_beginFileDownload(self, file): self._oprot.writeMessageBegin('beginFileDownload', TMessageType.CALL, self._seqid) args = beginFileDownload_args() args.file = file args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_beginFileDownload(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = beginFileDownload_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); def downloadChunk(self, id): """ Parameters: - id """ self.send_downloadChunk(id) return self.recv_downloadChunk() def send_downloadChunk(self, id): self._oprot.writeMessageBegin('downloadChunk', TMessageType.CALL, self._seqid) args = downloadChunk_args() args.id = id args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_downloadChunk(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = downloadChunk_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); def getNimbusConf(self, ): self.send_getNimbusConf() return self.recv_getNimbusConf() def send_getNimbusConf(self, ): self._oprot.writeMessageBegin('getNimbusConf', TMessageType.CALL, self._seqid) args = getNimbusConf_args() args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_getNimbusConf(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = getNimbusConf_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); def getClusterInfo(self, ): self.send_getClusterInfo() return self.recv_getClusterInfo() def send_getClusterInfo(self, ): self._oprot.writeMessageBegin('getClusterInfo', TMessageType.CALL, self._seqid) args = getClusterInfo_args() args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_getClusterInfo(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = getClusterInfo_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); def getTopologyInfo(self, id): """ Parameters: - id """ self.send_getTopologyInfo(id) return self.recv_getTopologyInfo() def send_getTopologyInfo(self, id): self._oprot.writeMessageBegin('getTopologyInfo', TMessageType.CALL, self._seqid) args = getTopologyInfo_args() args.id = id args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_getTopologyInfo(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = getTopologyInfo_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.e is not None: raise result.e raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); def getTopologyConf(self, id): """ Parameters: - id """ self.send_getTopologyConf(id) return self.recv_getTopologyConf() def send_getTopologyConf(self, id): self._oprot.writeMessageBegin('getTopologyConf', TMessageType.CALL, self._seqid) args = getTopologyConf_args() args.id = id args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_getTopologyConf(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = getTopologyConf_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.e is not None: raise result.e raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); def getTopology(self, id): """ Parameters: - id """ self.send_getTopology(id) return self.recv_getTopology() def send_getTopology(self, id): self._oprot.writeMessageBegin('getTopology', TMessageType.CALL, self._seqid) args = getTopology_args() args.id = id args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_getTopology(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = getTopology_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.e is not None: raise result.e raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); def getUserTopology(self, id): """ Parameters: - id """ self.send_getUserTopology(id) return self.recv_getUserTopology() def send_getUserTopology(self, id): self._oprot.writeMessageBegin('getUserTopology', TMessageType.CALL, self._seqid) args = getUserTopology_args() args.id = id args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() def recv_getUserTopology(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x result = getUserTopology_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.e is not None: raise result.e raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); class Processor(Iface, TProcessor): def __init__(self, handler): self._handler = handler self._processMap = {} self._processMap["submitTopology"] = Processor.process_submitTopology self._processMap["submitTopologyWithOpts"] = Processor.process_submitTopologyWithOpts self._processMap["killTopology"] = Processor.process_killTopology self._processMap["killTopologyWithOpts"] = Processor.process_killTopologyWithOpts self._processMap["activate"] = Processor.process_activate self._processMap["deactivate"] = Processor.process_deactivate self._processMap["rebalance"] = Processor.process_rebalance self._processMap["beginFileUpload"] = Processor.process_beginFileUpload self._processMap["uploadChunk"] = Processor.process_uploadChunk self._processMap["finishFileUpload"] = Processor.process_finishFileUpload self._processMap["beginFileDownload"] = Processor.process_beginFileDownload self._processMap["downloadChunk"] = Processor.process_downloadChunk self._processMap["getNimbusConf"] = Processor.process_getNimbusConf self._processMap["getClusterInfo"] = Processor.process_getClusterInfo self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo self._processMap["getTopologyConf"] = Processor.process_getTopologyConf self._processMap["getTopology"] = Processor.process_getTopology self._processMap["getUserTopology"] = Processor.process_getUserTopology def process(self, iprot, oprot): (name, type, seqid) = iprot.readMessageBegin() if name not in self._processMap: iprot.skip(TType.STRUCT) iprot.readMessageEnd() x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) x.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() return else: self._processMap[name](self, seqid, iprot, oprot) return True def process_submitTopology(self, seqid, iprot, oprot): args = submitTopology_args() args.read(iprot) iprot.readMessageEnd() result = submitTopology_result() try: self._handler.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology) except AlreadyAliveException, e: result.e = e except InvalidTopologyException, ite: result.ite = ite oprot.writeMessageBegin("submitTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_submitTopologyWithOpts(self, seqid, iprot, oprot): args = submitTopologyWithOpts_args() args.read(iprot) iprot.readMessageEnd() result = submitTopologyWithOpts_result() try: self._handler.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options) except AlreadyAliveException, e: result.e = e except InvalidTopologyException, ite: result.ite = ite oprot.writeMessageBegin("submitTopologyWithOpts", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_killTopology(self, seqid, iprot, oprot): args = killTopology_args() args.read(iprot) iprot.readMessageEnd() result = killTopology_result() try: self._handler.killTopology(args.name) except NotAliveException, e: result.e = e oprot.writeMessageBegin("killTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_killTopologyWithOpts(self, seqid, iprot, oprot): args = killTopologyWithOpts_args() args.read(iprot) iprot.readMessageEnd() result = killTopologyWithOpts_result() try: self._handler.killTopologyWithOpts(args.name, args.options) except NotAliveException, e: result.e = e oprot.writeMessageBegin("killTopologyWithOpts", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_activate(self, seqid, iprot, oprot): args = activate_args() args.read(iprot) iprot.readMessageEnd() result = activate_result() try: self._handler.activate(args.name) except NotAliveException, e: result.e = e oprot.writeMessageBegin("activate", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_deactivate(self, seqid, iprot, oprot): args = deactivate_args() args.read(iprot) iprot.readMessageEnd() result = deactivate_result() try: self._handler.deactivate(args.name) except NotAliveException, e: result.e = e oprot.writeMessageBegin("deactivate", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_rebalance(self, seqid, iprot, oprot): args = rebalance_args() args.read(iprot) iprot.readMessageEnd() result = rebalance_result() try: self._handler.rebalance(args.name, args.options) except NotAliveException, e: result.e = e except InvalidTopologyException, ite: result.ite = ite oprot.writeMessageBegin("rebalance", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_beginFileUpload(self, seqid, iprot, oprot): args = beginFileUpload_args() args.read(iprot) iprot.readMessageEnd() result = beginFileUpload_result() result.success = self._handler.beginFileUpload() oprot.writeMessageBegin("beginFileUpload", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_uploadChunk(self, seqid, iprot, oprot): args = uploadChunk_args() args.read(iprot) iprot.readMessageEnd() result = uploadChunk_result() self._handler.uploadChunk(args.location, args.chunk) oprot.writeMessageBegin("uploadChunk", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_finishFileUpload(self, seqid, iprot, oprot): args = finishFileUpload_args() args.read(iprot) iprot.readMessageEnd() result = finishFileUpload_result() self._handler.finishFileUpload(args.location) oprot.writeMessageBegin("finishFileUpload", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_beginFileDownload(self, seqid, iprot, oprot): args = beginFileDownload_args() args.read(iprot) iprot.readMessageEnd() result = beginFileDownload_result() result.success = self._handler.beginFileDownload(args.file) oprot.writeMessageBegin("beginFileDownload", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_downloadChunk(self, seqid, iprot, oprot): args = downloadChunk_args() args.read(iprot) iprot.readMessageEnd() result = downloadChunk_result() result.success = self._handler.downloadChunk(args.id) oprot.writeMessageBegin("downloadChunk", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_getNimbusConf(self, seqid, iprot, oprot): args = getNimbusConf_args() args.read(iprot) iprot.readMessageEnd() result = getNimbusConf_result() result.success = self._handler.getNimbusConf() oprot.writeMessageBegin("getNimbusConf", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_getClusterInfo(self, seqid, iprot, oprot): args = getClusterInfo_args() args.read(iprot) iprot.readMessageEnd() result = getClusterInfo_result() result.success = self._handler.getClusterInfo() oprot.writeMessageBegin("getClusterInfo", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_getTopologyInfo(self, seqid, iprot, oprot): args = getTopologyInfo_args() args.read(iprot) iprot.readMessageEnd() result = getTopologyInfo_result() try: result.success = self._handler.getTopologyInfo(args.id) except NotAliveException, e: result.e = e oprot.writeMessageBegin("getTopologyInfo", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_getTopologyConf(self, seqid, iprot, oprot): args = getTopologyConf_args() args.read(iprot) iprot.readMessageEnd() result = getTopologyConf_result() try: result.success = self._handler.getTopologyConf(args.id) except NotAliveException, e: result.e = e oprot.writeMessageBegin("getTopologyConf", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_getTopology(self, seqid, iprot, oprot): args = getTopology_args() args.read(iprot) iprot.readMessageEnd() result = getTopology_result() try: result.success = self._handler.getTopology(args.id) except NotAliveException, e: result.e = e oprot.writeMessageBegin("getTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() def process_getUserTopology(self, seqid, iprot, oprot): args = getUserTopology_args() args.read(iprot) iprot.readMessageEnd() result = getUserTopology_result() try: result.success = self._handler.getUserTopology(args.id) except NotAliveException, e: result.e = e oprot.writeMessageBegin("getUserTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() # HELPER FUNCTIONS AND STRUCTURES class submitTopology_args: """ Attributes: - name - uploadedJarLocation - jsonConf - topology """ thrift_spec = ( None, # 0 (1, TType.STRING, 'name', None, None, ), # 1 (2, TType.STRING, 'uploadedJarLocation', None, None, ), # 2 (3, TType.STRING, 'jsonConf', None, None, ), # 3 (4, TType.STRUCT, 'topology', (StormTopology, StormTopology.thrift_spec), None, ), # 4 ) def __hash__(self): return 0 + hash(self.name) + hash(self.uploadedJarLocation) + hash(self.jsonConf) + hash(self.topology) def __init__(self, name=None, uploadedJarLocation=None, jsonConf=None, topology=None,): self.name = name self.uploadedJarLocation = uploadedJarLocation self.jsonConf = jsonConf self.topology = topology def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.uploadedJarLocation = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: self.jsonConf = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: self.topology = StormTopology() self.topology.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('submitTopology_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() if self.uploadedJarLocation is not None: oprot.writeFieldBegin('uploadedJarLocation', TType.STRING, 2) oprot.writeString(self.uploadedJarLocation.encode('utf-8')) oprot.writeFieldEnd() if self.jsonConf is not None: oprot.writeFieldBegin('jsonConf', TType.STRING, 3) oprot.writeString(self.jsonConf.encode('utf-8')) oprot.writeFieldEnd() if self.topology is not None: oprot.writeFieldBegin('topology', TType.STRUCT, 4) self.topology.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class submitTopology_result: """ Attributes: - e - ite """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.e) + hash(self.ite) def __init__(self, e=None, ite=None,): self.e = e self.ite = ite def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.e = AlreadyAliveException() self.e.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.ite = InvalidTopologyException() self.ite.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('submitTopology_result') if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() if self.ite is not None: oprot.writeFieldBegin('ite', TType.STRUCT, 2) self.ite.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class submitTopologyWithOpts_args: """ Attributes: - name - uploadedJarLocation - jsonConf - topology - options """ thrift_spec = ( None, # 0 (1, TType.STRING, 'name', None, None, ), # 1 (2, TType.STRING, 'uploadedJarLocation', None, None, ), # 2 (3, TType.STRING, 'jsonConf', None, None, ), # 3 (4, TType.STRUCT, 'topology', (StormTopology, StormTopology.thrift_spec), None, ), # 4 (5, TType.STRUCT, 'options', (SubmitOptions, SubmitOptions.thrift_spec), None, ), # 5 ) def __hash__(self): return 0 + hash(self.name) + hash(self.uploadedJarLocation) + hash(self.jsonConf) + hash(self.topology) + hash(self.options) def __init__(self, name=None, uploadedJarLocation=None, jsonConf=None, topology=None, options=None,): self.name = name self.uploadedJarLocation = uploadedJarLocation self.jsonConf = jsonConf self.topology = topology self.options = options def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.uploadedJarLocation = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: self.jsonConf = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: self.topology = StormTopology() self.topology.read(iprot) else: iprot.skip(ftype) elif fid == 5: if ftype == TType.STRUCT: self.options = SubmitOptions() self.options.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('submitTopologyWithOpts_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() if self.uploadedJarLocation is not None: oprot.writeFieldBegin('uploadedJarLocation', TType.STRING, 2) oprot.writeString(self.uploadedJarLocation.encode('utf-8')) oprot.writeFieldEnd() if self.jsonConf is not None: oprot.writeFieldBegin('jsonConf', TType.STRING, 3) oprot.writeString(self.jsonConf.encode('utf-8')) oprot.writeFieldEnd() if self.topology is not None: oprot.writeFieldBegin('topology', TType.STRUCT, 4) self.topology.write(oprot) oprot.writeFieldEnd() if self.options is not None: oprot.writeFieldBegin('options', TType.STRUCT, 5) self.options.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class submitTopologyWithOpts_result: """ Attributes: - e - ite """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.e) + hash(self.ite) def __init__(self, e=None, ite=None,): self.e = e self.ite = ite def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.e = AlreadyAliveException() self.e.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.ite = InvalidTopologyException() self.ite.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('submitTopologyWithOpts_result') if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() if self.ite is not None: oprot.writeFieldBegin('ite', TType.STRUCT, 2) self.ite.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class killTopology_args: """ Attributes: - name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'name', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.name) def __init__(self, name=None,): self.name = name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('killTopology_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class killTopology_result: """ Attributes: - e """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.e) def __init__(self, e=None,): self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('killTopology_result') if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class killTopologyWithOpts_args: """ Attributes: - name - options """ thrift_spec = ( None, # 0 (1, TType.STRING, 'name', None, None, ), # 1 (2, TType.STRUCT, 'options', (KillOptions, KillOptions.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.name) + hash(self.options) def __init__(self, name=None, options=None,): self.name = name self.options = options def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.options = KillOptions() self.options.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('killTopologyWithOpts_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() if self.options is not None: oprot.writeFieldBegin('options', TType.STRUCT, 2) self.options.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class killTopologyWithOpts_result: """ Attributes: - e """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.e) def __init__(self, e=None,): self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('killTopologyWithOpts_result') if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class activate_args: """ Attributes: - name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'name', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.name) def __init__(self, name=None,): self.name = name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('activate_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class activate_result: """ Attributes: - e """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.e) def __init__(self, e=None,): self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('activate_result') if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class deactivate_args: """ Attributes: - name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'name', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.name) def __init__(self, name=None,): self.name = name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('deactivate_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class deactivate_result: """ Attributes: - e """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.e) def __init__(self, e=None,): self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('deactivate_result') if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class rebalance_args: """ Attributes: - name - options """ thrift_spec = ( None, # 0 (1, TType.STRING, 'name', None, None, ), # 1 (2, TType.STRUCT, 'options', (RebalanceOptions, RebalanceOptions.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.name) + hash(self.options) def __init__(self, name=None, options=None,): self.name = name self.options = options def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.options = RebalanceOptions() self.options.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('rebalance_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() if self.options is not None: oprot.writeFieldBegin('options', TType.STRUCT, 2) self.options.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class rebalance_result: """ Attributes: - e - ite """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.e) + hash(self.ite) def __init__(self, e=None, ite=None,): self.e = e self.ite = ite def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.ite = InvalidTopologyException() self.ite.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('rebalance_result') if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() if self.ite is not None: oprot.writeFieldBegin('ite', TType.STRUCT, 2) self.ite.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class beginFileUpload_args: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('beginFileUpload_args') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class beginFileUpload_result: """ Attributes: - success """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 ) def __hash__(self): return 0 + hash(self.success) def __init__(self, success=None,): self.success = success def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRING: self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('beginFileUpload_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class uploadChunk_args: """ Attributes: - location - chunk """ thrift_spec = ( None, # 0 (1, TType.STRING, 'location', None, None, ), # 1 (2, TType.STRING, 'chunk', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.location) + hash(self.chunk) def __init__(self, location=None, chunk=None,): self.location = location self.chunk = chunk def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.location = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.chunk = iprot.readString(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('uploadChunk_args') if self.location is not None: oprot.writeFieldBegin('location', TType.STRING, 1) oprot.writeString(self.location.encode('utf-8')) oprot.writeFieldEnd() if self.chunk is not None: oprot.writeFieldBegin('chunk', TType.STRING, 2) oprot.writeString(self.chunk) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class uploadChunk_result: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('uploadChunk_result') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class finishFileUpload_args: """ Attributes: - location """ thrift_spec = ( None, # 0 (1, TType.STRING, 'location', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.location) def __init__(self, location=None,): self.location = location def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.location = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('finishFileUpload_args') if self.location is not None: oprot.writeFieldBegin('location', TType.STRING, 1) oprot.writeString(self.location.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class finishFileUpload_result: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('finishFileUpload_result') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class beginFileDownload_args: """ Attributes: - file """ thrift_spec = ( None, # 0 (1, TType.STRING, 'file', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.file) def __init__(self, file=None,): self.file = file def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.file = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('beginFileDownload_args') if self.file is not None: oprot.writeFieldBegin('file', TType.STRING, 1) oprot.writeString(self.file.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class beginFileDownload_result: """ Attributes: - success """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 ) def __hash__(self): return 0 + hash(self.success) def __init__(self, success=None,): self.success = success def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRING: self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('beginFileDownload_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class downloadChunk_args: """ Attributes: - id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.id) def __init__(self, id=None,): self.id = id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('downloadChunk_args') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class downloadChunk_result: """ Attributes: - success """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 ) def __hash__(self): return 0 + hash(self.success) def __init__(self, success=None,): self.success = success def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRING: self.success = iprot.readString(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('downloadChunk_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getNimbusConf_args: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getNimbusConf_args') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getNimbusConf_result: """ Attributes: - success """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 ) def __hash__(self): return 0 + hash(self.success) def __init__(self, success=None,): self.success = success def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRING: self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getNimbusConf_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getClusterInfo_args: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getClusterInfo_args') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getClusterInfo_result: """ Attributes: - success """ thrift_spec = ( (0, TType.STRUCT, 'success', (ClusterSummary, ClusterSummary.thrift_spec), None, ), # 0 ) def __hash__(self): return 0 + hash(self.success) def __init__(self, success=None,): self.success = success def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRUCT: self.success = ClusterSummary() self.success.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getClusterInfo_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getTopologyInfo_args: """ Attributes: - id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.id) def __init__(self, id=None,): self.id = id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getTopologyInfo_args') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getTopologyInfo_result: """ Attributes: - success - e """ thrift_spec = ( (0, TType.STRUCT, 'success', (TopologyInfo, TopologyInfo.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.success) + hash(self.e) def __init__(self, success=None, e=None,): self.success = success self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRUCT: self.success = TopologyInfo() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getTopologyInfo_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) oprot.writeFieldEnd() if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getTopologyConf_args: """ Attributes: - id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.id) def __init__(self, id=None,): self.id = id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getTopologyConf_args') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getTopologyConf_result: """ Attributes: - success - e """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.success) + hash(self.e) def __init__(self, success=None, e=None,): self.success = success self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRING: self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getTopologyConf_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getTopology_args: """ Attributes: - id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.id) def __init__(self, id=None,): self.id = id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getTopology_args') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getTopology_result: """ Attributes: - success - e """ thrift_spec = ( (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.success) + hash(self.e) def __init__(self, success=None, e=None,): self.success = success self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRUCT: self.success = StormTopology() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getTopology_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) oprot.writeFieldEnd() if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getUserTopology_args: """ Attributes: - id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.id) def __init__(self, id=None,): self.id = id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getUserTopology_args') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class getUserTopology_result: """ Attributes: - success - e """ thrift_spec = ( (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 ) def __hash__(self): return 0 + hash(self.success) + hash(self.e) def __init__(self, success=None, e=None,): self.success = success self.e = e def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 0: if ftype == TType.STRUCT: self.success = StormTopology() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: self.e = NotAliveException() self.e.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('getUserTopology_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) oprot.writeFieldEnd() if self.e is not None: oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) ================================================ FILE: storm-core/src/py/storm/__init__.py ================================================ __all__ = ['ttypes', 'constants', 'Nimbus', 'DistributedRPC', 'DistributedRPCInvocations'] ================================================ FILE: storm-core/src/py/storm/constants.py ================================================ # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # from thrift.Thrift import * from ttypes import * ================================================ FILE: storm-core/src/py/storm/ttypes.py ================================================ # # Autogenerated by Thrift Compiler (0.7.0) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # from thrift.Thrift import * from thrift.transport import TTransport from thrift.protocol import TBinaryProtocol, TProtocol try: from thrift.protocol import fastbinary except: fastbinary = None class TopologyInitialStatus: ACTIVE = 1 INACTIVE = 2 _VALUES_TO_NAMES = { 1: "ACTIVE", 2: "INACTIVE", } _NAMES_TO_VALUES = { "ACTIVE": 1, "INACTIVE": 2, } class JavaObjectArg: """ Attributes: - int_arg - long_arg - string_arg - bool_arg - binary_arg - double_arg """ thrift_spec = ( None, # 0 (1, TType.I32, 'int_arg', None, None, ), # 1 (2, TType.I64, 'long_arg', None, None, ), # 2 (3, TType.STRING, 'string_arg', None, None, ), # 3 (4, TType.BOOL, 'bool_arg', None, None, ), # 4 (5, TType.STRING, 'binary_arg', None, None, ), # 5 (6, TType.DOUBLE, 'double_arg', None, None, ), # 6 ) def __hash__(self): return 0 + hash(self.int_arg) + hash(self.long_arg) + hash(self.string_arg) + hash(self.bool_arg) + hash(self.binary_arg) + hash(self.double_arg) def __init__(self, int_arg=None, long_arg=None, string_arg=None, bool_arg=None, binary_arg=None, double_arg=None,): self.int_arg = int_arg self.long_arg = long_arg self.string_arg = string_arg self.bool_arg = bool_arg self.binary_arg = binary_arg self.double_arg = double_arg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.I32: self.int_arg = iprot.readI32(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.I64: self.long_arg = iprot.readI64(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: self.string_arg = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 4: if ftype == TType.BOOL: self.bool_arg = iprot.readBool(); else: iprot.skip(ftype) elif fid == 5: if ftype == TType.STRING: self.binary_arg = iprot.readString(); else: iprot.skip(ftype) elif fid == 6: if ftype == TType.DOUBLE: self.double_arg = iprot.readDouble(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('JavaObjectArg') if self.int_arg is not None: oprot.writeFieldBegin('int_arg', TType.I32, 1) oprot.writeI32(self.int_arg) oprot.writeFieldEnd() if self.long_arg is not None: oprot.writeFieldBegin('long_arg', TType.I64, 2) oprot.writeI64(self.long_arg) oprot.writeFieldEnd() if self.string_arg is not None: oprot.writeFieldBegin('string_arg', TType.STRING, 3) oprot.writeString(self.string_arg.encode('utf-8')) oprot.writeFieldEnd() if self.bool_arg is not None: oprot.writeFieldBegin('bool_arg', TType.BOOL, 4) oprot.writeBool(self.bool_arg) oprot.writeFieldEnd() if self.binary_arg is not None: oprot.writeFieldBegin('binary_arg', TType.STRING, 5) oprot.writeString(self.binary_arg) oprot.writeFieldEnd() if self.double_arg is not None: oprot.writeFieldBegin('double_arg', TType.DOUBLE, 6) oprot.writeDouble(self.double_arg) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class JavaObject: """ Attributes: - full_class_name - args_list """ thrift_spec = ( None, # 0 (1, TType.STRING, 'full_class_name', None, None, ), # 1 (2, TType.LIST, 'args_list', (TType.STRUCT,(JavaObjectArg, JavaObjectArg.thrift_spec)), None, ), # 2 ) def __hash__(self): return 0 + hash(self.full_class_name) + hash(self.args_list) def __init__(self, full_class_name=None, args_list=None,): self.full_class_name = full_class_name self.args_list = args_list def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.full_class_name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.LIST: self.args_list = [] (_etype3, _size0) = iprot.readListBegin() for _i4 in xrange(_size0): _elem5 = JavaObjectArg() _elem5.read(iprot) self.args_list.append(_elem5) iprot.readListEnd() else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('JavaObject') if self.full_class_name is not None: oprot.writeFieldBegin('full_class_name', TType.STRING, 1) oprot.writeString(self.full_class_name.encode('utf-8')) oprot.writeFieldEnd() if self.args_list is not None: oprot.writeFieldBegin('args_list', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.args_list)) for iter6 in self.args_list: iter6.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.full_class_name is None: raise TProtocol.TProtocolException(message='Required field full_class_name is unset!') if self.args_list is None: raise TProtocol.TProtocolException(message='Required field args_list is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class NullStruct: thrift_spec = ( ) def __hash__(self): return 0 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('NullStruct') oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class GlobalStreamId: """ Attributes: - componentId - streamId """ thrift_spec = ( None, # 0 (1, TType.STRING, 'componentId', None, None, ), # 1 (2, TType.STRING, 'streamId', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.componentId) + hash(self.streamId) def __init__(self, componentId=None, streamId=None,): self.componentId = componentId self.streamId = streamId def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.componentId = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.streamId = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('GlobalStreamId') if self.componentId is not None: oprot.writeFieldBegin('componentId', TType.STRING, 1) oprot.writeString(self.componentId.encode('utf-8')) oprot.writeFieldEnd() if self.streamId is not None: oprot.writeFieldBegin('streamId', TType.STRING, 2) oprot.writeString(self.streamId.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.componentId is None: raise TProtocol.TProtocolException(message='Required field componentId is unset!') if self.streamId is None: raise TProtocol.TProtocolException(message='Required field streamId is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class Grouping: """ Attributes: - fields - shuffle - all - none - direct - custom_object - custom_serialized - local_or_shuffle """ thrift_spec = ( None, # 0 (1, TType.LIST, 'fields', (TType.STRING,None), None, ), # 1 (2, TType.STRUCT, 'shuffle', (NullStruct, NullStruct.thrift_spec), None, ), # 2 (3, TType.STRUCT, 'all', (NullStruct, NullStruct.thrift_spec), None, ), # 3 (4, TType.STRUCT, 'none', (NullStruct, NullStruct.thrift_spec), None, ), # 4 (5, TType.STRUCT, 'direct', (NullStruct, NullStruct.thrift_spec), None, ), # 5 (6, TType.STRUCT, 'custom_object', (JavaObject, JavaObject.thrift_spec), None, ), # 6 (7, TType.STRING, 'custom_serialized', None, None, ), # 7 (8, TType.STRUCT, 'local_or_shuffle', (NullStruct, NullStruct.thrift_spec), None, ), # 8 ) def __hash__(self): return 0 + hash(self.fields) + hash(self.shuffle) + hash(self.all) + hash(self.none) + hash(self.direct) + hash(self.custom_object) + hash(self.custom_serialized) + hash(self.local_or_shuffle) def __init__(self, fields=None, shuffle=None, all=None, none=None, direct=None, custom_object=None, custom_serialized=None, local_or_shuffle=None,): self.fields = fields self.shuffle = shuffle self.all = all self.none = none self.direct = direct self.custom_object = custom_object self.custom_serialized = custom_serialized self.local_or_shuffle = local_or_shuffle def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.LIST: self.fields = [] (_etype10, _size7) = iprot.readListBegin() for _i11 in xrange(_size7): _elem12 = iprot.readString().decode('utf-8') self.fields.append(_elem12) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.shuffle = NullStruct() self.shuffle.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: self.all = NullStruct() self.all.read(iprot) else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: self.none = NullStruct() self.none.read(iprot) else: iprot.skip(ftype) elif fid == 5: if ftype == TType.STRUCT: self.direct = NullStruct() self.direct.read(iprot) else: iprot.skip(ftype) elif fid == 6: if ftype == TType.STRUCT: self.custom_object = JavaObject() self.custom_object.read(iprot) else: iprot.skip(ftype) elif fid == 7: if ftype == TType.STRING: self.custom_serialized = iprot.readString(); else: iprot.skip(ftype) elif fid == 8: if ftype == TType.STRUCT: self.local_or_shuffle = NullStruct() self.local_or_shuffle.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('Grouping') if self.fields is not None: oprot.writeFieldBegin('fields', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.fields)) for iter13 in self.fields: oprot.writeString(iter13.encode('utf-8')) oprot.writeListEnd() oprot.writeFieldEnd() if self.shuffle is not None: oprot.writeFieldBegin('shuffle', TType.STRUCT, 2) self.shuffle.write(oprot) oprot.writeFieldEnd() if self.all is not None: oprot.writeFieldBegin('all', TType.STRUCT, 3) self.all.write(oprot) oprot.writeFieldEnd() if self.none is not None: oprot.writeFieldBegin('none', TType.STRUCT, 4) self.none.write(oprot) oprot.writeFieldEnd() if self.direct is not None: oprot.writeFieldBegin('direct', TType.STRUCT, 5) self.direct.write(oprot) oprot.writeFieldEnd() if self.custom_object is not None: oprot.writeFieldBegin('custom_object', TType.STRUCT, 6) self.custom_object.write(oprot) oprot.writeFieldEnd() if self.custom_serialized is not None: oprot.writeFieldBegin('custom_serialized', TType.STRING, 7) oprot.writeString(self.custom_serialized) oprot.writeFieldEnd() if self.local_or_shuffle is not None: oprot.writeFieldBegin('local_or_shuffle', TType.STRUCT, 8) self.local_or_shuffle.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class StreamInfo: """ Attributes: - output_fields - direct """ thrift_spec = ( None, # 0 (1, TType.LIST, 'output_fields', (TType.STRING,None), None, ), # 1 (2, TType.BOOL, 'direct', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.output_fields) + hash(self.direct) def __init__(self, output_fields=None, direct=None,): self.output_fields = output_fields self.direct = direct def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.LIST: self.output_fields = [] (_etype17, _size14) = iprot.readListBegin() for _i18 in xrange(_size14): _elem19 = iprot.readString().decode('utf-8') self.output_fields.append(_elem19) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.BOOL: self.direct = iprot.readBool(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('StreamInfo') if self.output_fields is not None: oprot.writeFieldBegin('output_fields', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.output_fields)) for iter20 in self.output_fields: oprot.writeString(iter20.encode('utf-8')) oprot.writeListEnd() oprot.writeFieldEnd() if self.direct is not None: oprot.writeFieldBegin('direct', TType.BOOL, 2) oprot.writeBool(self.direct) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.output_fields is None: raise TProtocol.TProtocolException(message='Required field output_fields is unset!') if self.direct is None: raise TProtocol.TProtocolException(message='Required field direct is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ShellComponent: """ Attributes: - execution_command - script """ thrift_spec = ( None, # 0 (1, TType.STRING, 'execution_command', None, None, ), # 1 (2, TType.STRING, 'script', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.execution_command) + hash(self.script) def __init__(self, execution_command=None, script=None,): self.execution_command = execution_command self.script = script def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.execution_command = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.script = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ShellComponent') if self.execution_command is not None: oprot.writeFieldBegin('execution_command', TType.STRING, 1) oprot.writeString(self.execution_command.encode('utf-8')) oprot.writeFieldEnd() if self.script is not None: oprot.writeFieldBegin('script', TType.STRING, 2) oprot.writeString(self.script.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ComponentObject: """ Attributes: - serialized_java - shell - java_object """ thrift_spec = ( None, # 0 (1, TType.STRING, 'serialized_java', None, None, ), # 1 (2, TType.STRUCT, 'shell', (ShellComponent, ShellComponent.thrift_spec), None, ), # 2 (3, TType.STRUCT, 'java_object', (JavaObject, JavaObject.thrift_spec), None, ), # 3 ) def __hash__(self): return 0 + hash(self.serialized_java) + hash(self.shell) + hash(self.java_object) def __init__(self, serialized_java=None, shell=None, java_object=None,): self.serialized_java = serialized_java self.shell = shell self.java_object = java_object def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.serialized_java = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.shell = ShellComponent() self.shell.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: self.java_object = JavaObject() self.java_object.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ComponentObject') if self.serialized_java is not None: oprot.writeFieldBegin('serialized_java', TType.STRING, 1) oprot.writeString(self.serialized_java) oprot.writeFieldEnd() if self.shell is not None: oprot.writeFieldBegin('shell', TType.STRUCT, 2) self.shell.write(oprot) oprot.writeFieldEnd() if self.java_object is not None: oprot.writeFieldBegin('java_object', TType.STRUCT, 3) self.java_object.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ComponentCommon: """ Attributes: - inputs - streams - parallelism_hint - json_conf """ thrift_spec = ( None, # 0 (1, TType.MAP, 'inputs', (TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.STRUCT,(Grouping, Grouping.thrift_spec)), None, ), # 1 (2, TType.MAP, 'streams', (TType.STRING,None,TType.STRUCT,(StreamInfo, StreamInfo.thrift_spec)), None, ), # 2 (3, TType.I32, 'parallelism_hint', None, None, ), # 3 (4, TType.STRING, 'json_conf', None, None, ), # 4 ) def __hash__(self): return 0 + hash(self.inputs) + hash(self.streams) + hash(self.parallelism_hint) + hash(self.json_conf) def __init__(self, inputs=None, streams=None, parallelism_hint=None, json_conf=None,): self.inputs = inputs self.streams = streams self.parallelism_hint = parallelism_hint self.json_conf = json_conf def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.MAP: self.inputs = {} (_ktype22, _vtype23, _size21 ) = iprot.readMapBegin() for _i25 in xrange(_size21): _key26 = GlobalStreamId() _key26.read(iprot) _val27 = Grouping() _val27.read(iprot) self.inputs[_key26] = _val27 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.MAP: self.streams = {} (_ktype29, _vtype30, _size28 ) = iprot.readMapBegin() for _i32 in xrange(_size28): _key33 = iprot.readString().decode('utf-8') _val34 = StreamInfo() _val34.read(iprot) self.streams[_key33] = _val34 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.I32: self.parallelism_hint = iprot.readI32(); else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRING: self.json_conf = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ComponentCommon') if self.inputs is not None: oprot.writeFieldBegin('inputs', TType.MAP, 1) oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.inputs)) for kiter35,viter36 in self.inputs.items(): kiter35.write(oprot) viter36.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.streams is not None: oprot.writeFieldBegin('streams', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.streams)) for kiter37,viter38 in self.streams.items(): oprot.writeString(kiter37.encode('utf-8')) viter38.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.parallelism_hint is not None: oprot.writeFieldBegin('parallelism_hint', TType.I32, 3) oprot.writeI32(self.parallelism_hint) oprot.writeFieldEnd() if self.json_conf is not None: oprot.writeFieldBegin('json_conf', TType.STRING, 4) oprot.writeString(self.json_conf.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.inputs is None: raise TProtocol.TProtocolException(message='Required field inputs is unset!') if self.streams is None: raise TProtocol.TProtocolException(message='Required field streams is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class SpoutSpec: """ Attributes: - spout_object - common """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.spout_object) + hash(self.common) def __init__(self, spout_object=None, common=None,): self.spout_object = spout_object self.common = common def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.spout_object = ComponentObject() self.spout_object.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.common = ComponentCommon() self.common.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('SpoutSpec') if self.spout_object is not None: oprot.writeFieldBegin('spout_object', TType.STRUCT, 1) self.spout_object.write(oprot) oprot.writeFieldEnd() if self.common is not None: oprot.writeFieldBegin('common', TType.STRUCT, 2) self.common.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.spout_object is None: raise TProtocol.TProtocolException(message='Required field spout_object is unset!') if self.common is None: raise TProtocol.TProtocolException(message='Required field common is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class Bolt: """ Attributes: - bolt_object - common """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'bolt_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.bolt_object) + hash(self.common) def __init__(self, bolt_object=None, common=None,): self.bolt_object = bolt_object self.common = common def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.bolt_object = ComponentObject() self.bolt_object.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.common = ComponentCommon() self.common.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('Bolt') if self.bolt_object is not None: oprot.writeFieldBegin('bolt_object', TType.STRUCT, 1) self.bolt_object.write(oprot) oprot.writeFieldEnd() if self.common is not None: oprot.writeFieldBegin('common', TType.STRUCT, 2) self.common.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.bolt_object is None: raise TProtocol.TProtocolException(message='Required field bolt_object is unset!') if self.common is None: raise TProtocol.TProtocolException(message='Required field common is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class StateSpoutSpec: """ Attributes: - state_spout_object - common """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'state_spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.state_spout_object) + hash(self.common) def __init__(self, state_spout_object=None, common=None,): self.state_spout_object = state_spout_object self.common = common def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.state_spout_object = ComponentObject() self.state_spout_object.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.common = ComponentCommon() self.common.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('StateSpoutSpec') if self.state_spout_object is not None: oprot.writeFieldBegin('state_spout_object', TType.STRUCT, 1) self.state_spout_object.write(oprot) oprot.writeFieldEnd() if self.common is not None: oprot.writeFieldBegin('common', TType.STRUCT, 2) self.common.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.state_spout_object is None: raise TProtocol.TProtocolException(message='Required field state_spout_object is unset!') if self.common is None: raise TProtocol.TProtocolException(message='Required field common is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class StormTopology: """ Attributes: - spouts - bolts - state_spouts """ thrift_spec = ( None, # 0 (1, TType.MAP, 'spouts', (TType.STRING,None,TType.STRUCT,(SpoutSpec, SpoutSpec.thrift_spec)), None, ), # 1 (2, TType.MAP, 'bolts', (TType.STRING,None,TType.STRUCT,(Bolt, Bolt.thrift_spec)), None, ), # 2 (3, TType.MAP, 'state_spouts', (TType.STRING,None,TType.STRUCT,(StateSpoutSpec, StateSpoutSpec.thrift_spec)), None, ), # 3 ) def __hash__(self): return 0 + hash(self.spouts) + hash(self.bolts) + hash(self.state_spouts) def __init__(self, spouts=None, bolts=None, state_spouts=None,): self.spouts = spouts self.bolts = bolts self.state_spouts = state_spouts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.MAP: self.spouts = {} (_ktype40, _vtype41, _size39 ) = iprot.readMapBegin() for _i43 in xrange(_size39): _key44 = iprot.readString().decode('utf-8') _val45 = SpoutSpec() _val45.read(iprot) self.spouts[_key44] = _val45 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.MAP: self.bolts = {} (_ktype47, _vtype48, _size46 ) = iprot.readMapBegin() for _i50 in xrange(_size46): _key51 = iprot.readString().decode('utf-8') _val52 = Bolt() _val52.read(iprot) self.bolts[_key51] = _val52 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.MAP: self.state_spouts = {} (_ktype54, _vtype55, _size53 ) = iprot.readMapBegin() for _i57 in xrange(_size53): _key58 = iprot.readString().decode('utf-8') _val59 = StateSpoutSpec() _val59.read(iprot) self.state_spouts[_key58] = _val59 iprot.readMapEnd() else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('StormTopology') if self.spouts is not None: oprot.writeFieldBegin('spouts', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.spouts)) for kiter60,viter61 in self.spouts.items(): oprot.writeString(kiter60.encode('utf-8')) viter61.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.bolts is not None: oprot.writeFieldBegin('bolts', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.bolts)) for kiter62,viter63 in self.bolts.items(): oprot.writeString(kiter62.encode('utf-8')) viter63.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.state_spouts is not None: oprot.writeFieldBegin('state_spouts', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.state_spouts)) for kiter64,viter65 in self.state_spouts.items(): oprot.writeString(kiter64.encode('utf-8')) viter65.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.spouts is None: raise TProtocol.TProtocolException(message='Required field spouts is unset!') if self.bolts is None: raise TProtocol.TProtocolException(message='Required field bolts is unset!') if self.state_spouts is None: raise TProtocol.TProtocolException(message='Required field state_spouts is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class AlreadyAliveException(Exception): """ Attributes: - msg """ thrift_spec = ( None, # 0 (1, TType.STRING, 'msg', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.msg) def __init__(self, msg=None,): self.msg = msg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.msg = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('AlreadyAliveException') if self.msg is not None: oprot.writeFieldBegin('msg', TType.STRING, 1) oprot.writeString(self.msg.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.msg is None: raise TProtocol.TProtocolException(message='Required field msg is unset!') return def __str__(self): return repr(self) def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class NotAliveException(Exception): """ Attributes: - msg """ thrift_spec = ( None, # 0 (1, TType.STRING, 'msg', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.msg) def __init__(self, msg=None,): self.msg = msg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.msg = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('NotAliveException') if self.msg is not None: oprot.writeFieldBegin('msg', TType.STRING, 1) oprot.writeString(self.msg.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.msg is None: raise TProtocol.TProtocolException(message='Required field msg is unset!') return def __str__(self): return repr(self) def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class InvalidTopologyException(Exception): """ Attributes: - msg """ thrift_spec = ( None, # 0 (1, TType.STRING, 'msg', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.msg) def __init__(self, msg=None,): self.msg = msg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.msg = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('InvalidTopologyException') if self.msg is not None: oprot.writeFieldBegin('msg', TType.STRING, 1) oprot.writeString(self.msg.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.msg is None: raise TProtocol.TProtocolException(message='Required field msg is unset!') return def __str__(self): return repr(self) def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class TopologySummary: """ Attributes: - id - name - num_tasks - num_executors - num_workers - uptime_secs - status """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 (2, TType.STRING, 'name', None, None, ), # 2 (3, TType.I32, 'num_tasks', None, None, ), # 3 (4, TType.I32, 'num_executors', None, None, ), # 4 (5, TType.I32, 'num_workers', None, None, ), # 5 (6, TType.I32, 'uptime_secs', None, None, ), # 6 (7, TType.STRING, 'status', None, None, ), # 7 ) def __hash__(self): return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status) def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None,): self.id = id self.name = name self.num_tasks = num_tasks self.num_executors = num_executors self.num_workers = num_workers self.uptime_secs = uptime_secs self.status = status def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 3: if ftype == TType.I32: self.num_tasks = iprot.readI32(); else: iprot.skip(ftype) elif fid == 4: if ftype == TType.I32: self.num_executors = iprot.readI32(); else: iprot.skip(ftype) elif fid == 5: if ftype == TType.I32: self.num_workers = iprot.readI32(); else: iprot.skip(ftype) elif fid == 6: if ftype == TType.I32: self.uptime_secs = iprot.readI32(); else: iprot.skip(ftype) elif fid == 7: if ftype == TType.STRING: self.status = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('TopologySummary') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 2) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() if self.num_tasks is not None: oprot.writeFieldBegin('num_tasks', TType.I32, 3) oprot.writeI32(self.num_tasks) oprot.writeFieldEnd() if self.num_executors is not None: oprot.writeFieldBegin('num_executors', TType.I32, 4) oprot.writeI32(self.num_executors) oprot.writeFieldEnd() if self.num_workers is not None: oprot.writeFieldBegin('num_workers', TType.I32, 5) oprot.writeI32(self.num_workers) oprot.writeFieldEnd() if self.uptime_secs is not None: oprot.writeFieldBegin('uptime_secs', TType.I32, 6) oprot.writeI32(self.uptime_secs) oprot.writeFieldEnd() if self.status is not None: oprot.writeFieldBegin('status', TType.STRING, 7) oprot.writeString(self.status.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.id is None: raise TProtocol.TProtocolException(message='Required field id is unset!') if self.name is None: raise TProtocol.TProtocolException(message='Required field name is unset!') if self.num_tasks is None: raise TProtocol.TProtocolException(message='Required field num_tasks is unset!') if self.num_executors is None: raise TProtocol.TProtocolException(message='Required field num_executors is unset!') if self.num_workers is None: raise TProtocol.TProtocolException(message='Required field num_workers is unset!') if self.uptime_secs is None: raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') if self.status is None: raise TProtocol.TProtocolException(message='Required field status is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class SupervisorSummary: """ Attributes: - host - uptime_secs - num_workers - num_used_workers - supervisor_id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'host', None, None, ), # 1 (2, TType.I32, 'uptime_secs', None, None, ), # 2 (3, TType.I32, 'num_workers', None, None, ), # 3 (4, TType.I32, 'num_used_workers', None, None, ), # 4 (5, TType.STRING, 'supervisor_id', None, None, ), # 5 ) def __hash__(self): return 0 + hash(self.host) + hash(self.uptime_secs) + hash(self.num_workers) + hash(self.num_used_workers) + hash(self.supervisor_id) def __init__(self, host=None, uptime_secs=None, num_workers=None, num_used_workers=None, supervisor_id=None,): self.host = host self.uptime_secs = uptime_secs self.num_workers = num_workers self.num_used_workers = num_used_workers self.supervisor_id = supervisor_id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.host = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.I32: self.uptime_secs = iprot.readI32(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.I32: self.num_workers = iprot.readI32(); else: iprot.skip(ftype) elif fid == 4: if ftype == TType.I32: self.num_used_workers = iprot.readI32(); else: iprot.skip(ftype) elif fid == 5: if ftype == TType.STRING: self.supervisor_id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('SupervisorSummary') if self.host is not None: oprot.writeFieldBegin('host', TType.STRING, 1) oprot.writeString(self.host.encode('utf-8')) oprot.writeFieldEnd() if self.uptime_secs is not None: oprot.writeFieldBegin('uptime_secs', TType.I32, 2) oprot.writeI32(self.uptime_secs) oprot.writeFieldEnd() if self.num_workers is not None: oprot.writeFieldBegin('num_workers', TType.I32, 3) oprot.writeI32(self.num_workers) oprot.writeFieldEnd() if self.num_used_workers is not None: oprot.writeFieldBegin('num_used_workers', TType.I32, 4) oprot.writeI32(self.num_used_workers) oprot.writeFieldEnd() if self.supervisor_id is not None: oprot.writeFieldBegin('supervisor_id', TType.STRING, 5) oprot.writeString(self.supervisor_id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.host is None: raise TProtocol.TProtocolException(message='Required field host is unset!') if self.uptime_secs is None: raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') if self.num_workers is None: raise TProtocol.TProtocolException(message='Required field num_workers is unset!') if self.num_used_workers is None: raise TProtocol.TProtocolException(message='Required field num_used_workers is unset!') if self.supervisor_id is None: raise TProtocol.TProtocolException(message='Required field supervisor_id is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ClusterSummary: """ Attributes: - supervisors - nimbus_uptime_secs - topologies """ thrift_spec = ( None, # 0 (1, TType.LIST, 'supervisors', (TType.STRUCT,(SupervisorSummary, SupervisorSummary.thrift_spec)), None, ), # 1 (2, TType.I32, 'nimbus_uptime_secs', None, None, ), # 2 (3, TType.LIST, 'topologies', (TType.STRUCT,(TopologySummary, TopologySummary.thrift_spec)), None, ), # 3 ) def __hash__(self): return 0 + hash(self.supervisors) + hash(self.nimbus_uptime_secs) + hash(self.topologies) def __init__(self, supervisors=None, nimbus_uptime_secs=None, topologies=None,): self.supervisors = supervisors self.nimbus_uptime_secs = nimbus_uptime_secs self.topologies = topologies def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.LIST: self.supervisors = [] (_etype69, _size66) = iprot.readListBegin() for _i70 in xrange(_size66): _elem71 = SupervisorSummary() _elem71.read(iprot) self.supervisors.append(_elem71) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.I32: self.nimbus_uptime_secs = iprot.readI32(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.LIST: self.topologies = [] (_etype75, _size72) = iprot.readListBegin() for _i76 in xrange(_size72): _elem77 = TopologySummary() _elem77.read(iprot) self.topologies.append(_elem77) iprot.readListEnd() else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ClusterSummary') if self.supervisors is not None: oprot.writeFieldBegin('supervisors', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.supervisors)) for iter78 in self.supervisors: iter78.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.nimbus_uptime_secs is not None: oprot.writeFieldBegin('nimbus_uptime_secs', TType.I32, 2) oprot.writeI32(self.nimbus_uptime_secs) oprot.writeFieldEnd() if self.topologies is not None: oprot.writeFieldBegin('topologies', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.topologies)) for iter79 in self.topologies: iter79.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.supervisors is None: raise TProtocol.TProtocolException(message='Required field supervisors is unset!') if self.nimbus_uptime_secs is None: raise TProtocol.TProtocolException(message='Required field nimbus_uptime_secs is unset!') if self.topologies is None: raise TProtocol.TProtocolException(message='Required field topologies is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ErrorInfo: """ Attributes: - error - error_time_secs """ thrift_spec = ( None, # 0 (1, TType.STRING, 'error', None, None, ), # 1 (2, TType.I32, 'error_time_secs', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.error) + hash(self.error_time_secs) def __init__(self, error=None, error_time_secs=None,): self.error = error self.error_time_secs = error_time_secs def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.error = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.I32: self.error_time_secs = iprot.readI32(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ErrorInfo') if self.error is not None: oprot.writeFieldBegin('error', TType.STRING, 1) oprot.writeString(self.error.encode('utf-8')) oprot.writeFieldEnd() if self.error_time_secs is not None: oprot.writeFieldBegin('error_time_secs', TType.I32, 2) oprot.writeI32(self.error_time_secs) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.error is None: raise TProtocol.TProtocolException(message='Required field error is unset!') if self.error_time_secs is None: raise TProtocol.TProtocolException(message='Required field error_time_secs is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class BoltStats: """ Attributes: - acked - failed - process_ms_avg - executed - execute_ms_avg """ thrift_spec = ( None, # 0 (1, TType.MAP, 'acked', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 1 (2, TType.MAP, 'failed', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 2 (3, TType.MAP, 'process_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.DOUBLE,None)), None, ), # 3 (4, TType.MAP, 'executed', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 4 (5, TType.MAP, 'execute_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.DOUBLE,None)), None, ), # 5 ) def __hash__(self): return 0 + hash(self.acked) + hash(self.failed) + hash(self.process_ms_avg) + hash(self.executed) + hash(self.execute_ms_avg) def __init__(self, acked=None, failed=None, process_ms_avg=None, executed=None, execute_ms_avg=None,): self.acked = acked self.failed = failed self.process_ms_avg = process_ms_avg self.executed = executed self.execute_ms_avg = execute_ms_avg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.MAP: self.acked = {} (_ktype81, _vtype82, _size80 ) = iprot.readMapBegin() for _i84 in xrange(_size80): _key85 = iprot.readString().decode('utf-8') _val86 = {} (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin() for _i91 in xrange(_size87): _key92 = GlobalStreamId() _key92.read(iprot) _val93 = iprot.readI64(); _val86[_key92] = _val93 iprot.readMapEnd() self.acked[_key85] = _val86 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.MAP: self.failed = {} (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin() for _i98 in xrange(_size94): _key99 = iprot.readString().decode('utf-8') _val100 = {} (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin() for _i105 in xrange(_size101): _key106 = GlobalStreamId() _key106.read(iprot) _val107 = iprot.readI64(); _val100[_key106] = _val107 iprot.readMapEnd() self.failed[_key99] = _val100 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.MAP: self.process_ms_avg = {} (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin() for _i112 in xrange(_size108): _key113 = iprot.readString().decode('utf-8') _val114 = {} (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin() for _i119 in xrange(_size115): _key120 = GlobalStreamId() _key120.read(iprot) _val121 = iprot.readDouble(); _val114[_key120] = _val121 iprot.readMapEnd() self.process_ms_avg[_key113] = _val114 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.MAP: self.executed = {} (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin() for _i126 in xrange(_size122): _key127 = iprot.readString().decode('utf-8') _val128 = {} (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin() for _i133 in xrange(_size129): _key134 = GlobalStreamId() _key134.read(iprot) _val135 = iprot.readI64(); _val128[_key134] = _val135 iprot.readMapEnd() self.executed[_key127] = _val128 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.MAP: self.execute_ms_avg = {} (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin() for _i140 in xrange(_size136): _key141 = iprot.readString().decode('utf-8') _val142 = {} (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin() for _i147 in xrange(_size143): _key148 = GlobalStreamId() _key148.read(iprot) _val149 = iprot.readDouble(); _val142[_key148] = _val149 iprot.readMapEnd() self.execute_ms_avg[_key141] = _val142 iprot.readMapEnd() else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('BoltStats') if self.acked is not None: oprot.writeFieldBegin('acked', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked)) for kiter150,viter151 in self.acked.items(): oprot.writeString(kiter150.encode('utf-8')) oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter151)) for kiter152,viter153 in viter151.items(): kiter152.write(oprot) oprot.writeI64(viter153) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.failed is not None: oprot.writeFieldBegin('failed', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed)) for kiter154,viter155 in self.failed.items(): oprot.writeString(kiter154.encode('utf-8')) oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter155)) for kiter156,viter157 in viter155.items(): kiter156.write(oprot) oprot.writeI64(viter157) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.process_ms_avg is not None: oprot.writeFieldBegin('process_ms_avg', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.process_ms_avg)) for kiter158,viter159 in self.process_ms_avg.items(): oprot.writeString(kiter158.encode('utf-8')) oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter159)) for kiter160,viter161 in viter159.items(): kiter160.write(oprot) oprot.writeDouble(viter161) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.executed is not None: oprot.writeFieldBegin('executed', TType.MAP, 4) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.executed)) for kiter162,viter163 in self.executed.items(): oprot.writeString(kiter162.encode('utf-8')) oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter163)) for kiter164,viter165 in viter163.items(): kiter164.write(oprot) oprot.writeI64(viter165) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.execute_ms_avg is not None: oprot.writeFieldBegin('execute_ms_avg', TType.MAP, 5) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.execute_ms_avg)) for kiter166,viter167 in self.execute_ms_avg.items(): oprot.writeString(kiter166.encode('utf-8')) oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter167)) for kiter168,viter169 in viter167.items(): kiter168.write(oprot) oprot.writeDouble(viter169) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.acked is None: raise TProtocol.TProtocolException(message='Required field acked is unset!') if self.failed is None: raise TProtocol.TProtocolException(message='Required field failed is unset!') if self.process_ms_avg is None: raise TProtocol.TProtocolException(message='Required field process_ms_avg is unset!') if self.executed is None: raise TProtocol.TProtocolException(message='Required field executed is unset!') if self.execute_ms_avg is None: raise TProtocol.TProtocolException(message='Required field execute_ms_avg is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class SpoutStats: """ Attributes: - acked - failed - complete_ms_avg """ thrift_spec = ( None, # 0 (1, TType.MAP, 'acked', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 1 (2, TType.MAP, 'failed', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 2 (3, TType.MAP, 'complete_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.DOUBLE,None)), None, ), # 3 ) def __hash__(self): return 0 + hash(self.acked) + hash(self.failed) + hash(self.complete_ms_avg) def __init__(self, acked=None, failed=None, complete_ms_avg=None,): self.acked = acked self.failed = failed self.complete_ms_avg = complete_ms_avg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.MAP: self.acked = {} (_ktype171, _vtype172, _size170 ) = iprot.readMapBegin() for _i174 in xrange(_size170): _key175 = iprot.readString().decode('utf-8') _val176 = {} (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin() for _i181 in xrange(_size177): _key182 = iprot.readString().decode('utf-8') _val183 = iprot.readI64(); _val176[_key182] = _val183 iprot.readMapEnd() self.acked[_key175] = _val176 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.MAP: self.failed = {} (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin() for _i188 in xrange(_size184): _key189 = iprot.readString().decode('utf-8') _val190 = {} (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin() for _i195 in xrange(_size191): _key196 = iprot.readString().decode('utf-8') _val197 = iprot.readI64(); _val190[_key196] = _val197 iprot.readMapEnd() self.failed[_key189] = _val190 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.MAP: self.complete_ms_avg = {} (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin() for _i202 in xrange(_size198): _key203 = iprot.readString().decode('utf-8') _val204 = {} (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin() for _i209 in xrange(_size205): _key210 = iprot.readString().decode('utf-8') _val211 = iprot.readDouble(); _val204[_key210] = _val211 iprot.readMapEnd() self.complete_ms_avg[_key203] = _val204 iprot.readMapEnd() else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('SpoutStats') if self.acked is not None: oprot.writeFieldBegin('acked', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked)) for kiter212,viter213 in self.acked.items(): oprot.writeString(kiter212.encode('utf-8')) oprot.writeMapBegin(TType.STRING, TType.I64, len(viter213)) for kiter214,viter215 in viter213.items(): oprot.writeString(kiter214.encode('utf-8')) oprot.writeI64(viter215) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.failed is not None: oprot.writeFieldBegin('failed', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed)) for kiter216,viter217 in self.failed.items(): oprot.writeString(kiter216.encode('utf-8')) oprot.writeMapBegin(TType.STRING, TType.I64, len(viter217)) for kiter218,viter219 in viter217.items(): oprot.writeString(kiter218.encode('utf-8')) oprot.writeI64(viter219) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.complete_ms_avg is not None: oprot.writeFieldBegin('complete_ms_avg', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.complete_ms_avg)) for kiter220,viter221 in self.complete_ms_avg.items(): oprot.writeString(kiter220.encode('utf-8')) oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter221)) for kiter222,viter223 in viter221.items(): oprot.writeString(kiter222.encode('utf-8')) oprot.writeDouble(viter223) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.acked is None: raise TProtocol.TProtocolException(message='Required field acked is unset!') if self.failed is None: raise TProtocol.TProtocolException(message='Required field failed is unset!') if self.complete_ms_avg is None: raise TProtocol.TProtocolException(message='Required field complete_ms_avg is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ExecutorSpecificStats: """ Attributes: - bolt - spout """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'bolt', (BoltStats, BoltStats.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'spout', (SpoutStats, SpoutStats.thrift_spec), None, ), # 2 ) def __hash__(self): return 0 + hash(self.bolt) + hash(self.spout) def __init__(self, bolt=None, spout=None,): self.bolt = bolt self.spout = spout def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.bolt = BoltStats() self.bolt.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: self.spout = SpoutStats() self.spout.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ExecutorSpecificStats') if self.bolt is not None: oprot.writeFieldBegin('bolt', TType.STRUCT, 1) self.bolt.write(oprot) oprot.writeFieldEnd() if self.spout is not None: oprot.writeFieldBegin('spout', TType.STRUCT, 2) self.spout.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ExecutorStats: """ Attributes: - emitted - transferred - specific """ thrift_spec = ( None, # 0 (1, TType.MAP, 'emitted', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 1 (2, TType.MAP, 'transferred', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 2 (3, TType.STRUCT, 'specific', (ExecutorSpecificStats, ExecutorSpecificStats.thrift_spec), None, ), # 3 ) def __hash__(self): return 0 + hash(self.emitted) + hash(self.transferred) + hash(self.specific) def __init__(self, emitted=None, transferred=None, specific=None,): self.emitted = emitted self.transferred = transferred self.specific = specific def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.MAP: self.emitted = {} (_ktype225, _vtype226, _size224 ) = iprot.readMapBegin() for _i228 in xrange(_size224): _key229 = iprot.readString().decode('utf-8') _val230 = {} (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin() for _i235 in xrange(_size231): _key236 = iprot.readString().decode('utf-8') _val237 = iprot.readI64(); _val230[_key236] = _val237 iprot.readMapEnd() self.emitted[_key229] = _val230 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.MAP: self.transferred = {} (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin() for _i242 in xrange(_size238): _key243 = iprot.readString().decode('utf-8') _val244 = {} (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin() for _i249 in xrange(_size245): _key250 = iprot.readString().decode('utf-8') _val251 = iprot.readI64(); _val244[_key250] = _val251 iprot.readMapEnd() self.transferred[_key243] = _val244 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: self.specific = ExecutorSpecificStats() self.specific.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ExecutorStats') if self.emitted is not None: oprot.writeFieldBegin('emitted', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.emitted)) for kiter252,viter253 in self.emitted.items(): oprot.writeString(kiter252.encode('utf-8')) oprot.writeMapBegin(TType.STRING, TType.I64, len(viter253)) for kiter254,viter255 in viter253.items(): oprot.writeString(kiter254.encode('utf-8')) oprot.writeI64(viter255) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.transferred is not None: oprot.writeFieldBegin('transferred', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.transferred)) for kiter256,viter257 in self.transferred.items(): oprot.writeString(kiter256.encode('utf-8')) oprot.writeMapBegin(TType.STRING, TType.I64, len(viter257)) for kiter258,viter259 in viter257.items(): oprot.writeString(kiter258.encode('utf-8')) oprot.writeI64(viter259) oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() if self.specific is not None: oprot.writeFieldBegin('specific', TType.STRUCT, 3) self.specific.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.emitted is None: raise TProtocol.TProtocolException(message='Required field emitted is unset!') if self.transferred is None: raise TProtocol.TProtocolException(message='Required field transferred is unset!') if self.specific is None: raise TProtocol.TProtocolException(message='Required field specific is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ExecutorInfo: """ Attributes: - task_start - task_end """ thrift_spec = ( None, # 0 (1, TType.I32, 'task_start', None, None, ), # 1 (2, TType.I32, 'task_end', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.task_start) + hash(self.task_end) def __init__(self, task_start=None, task_end=None,): self.task_start = task_start self.task_end = task_end def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.I32: self.task_start = iprot.readI32(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.I32: self.task_end = iprot.readI32(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ExecutorInfo') if self.task_start is not None: oprot.writeFieldBegin('task_start', TType.I32, 1) oprot.writeI32(self.task_start) oprot.writeFieldEnd() if self.task_end is not None: oprot.writeFieldBegin('task_end', TType.I32, 2) oprot.writeI32(self.task_end) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.task_start is None: raise TProtocol.TProtocolException(message='Required field task_start is unset!') if self.task_end is None: raise TProtocol.TProtocolException(message='Required field task_end is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class ExecutorSummary: """ Attributes: - executor_info - component_id - host - port - uptime_secs - stats """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'executor_info', (ExecutorInfo, ExecutorInfo.thrift_spec), None, ), # 1 (2, TType.STRING, 'component_id', None, None, ), # 2 (3, TType.STRING, 'host', None, None, ), # 3 (4, TType.I32, 'port', None, None, ), # 4 (5, TType.I32, 'uptime_secs', None, None, ), # 5 None, # 6 (7, TType.STRUCT, 'stats', (ExecutorStats, ExecutorStats.thrift_spec), None, ), # 7 ) def __hash__(self): return 0 + hash(self.executor_info) + hash(self.component_id) + hash(self.host) + hash(self.port) + hash(self.uptime_secs) + hash(self.stats) def __init__(self, executor_info=None, component_id=None, host=None, port=None, uptime_secs=None, stats=None,): self.executor_info = executor_info self.component_id = component_id self.host = host self.port = port self.uptime_secs = uptime_secs self.stats = stats def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRUCT: self.executor_info = ExecutorInfo() self.executor_info.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.component_id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: self.host = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 4: if ftype == TType.I32: self.port = iprot.readI32(); else: iprot.skip(ftype) elif fid == 5: if ftype == TType.I32: self.uptime_secs = iprot.readI32(); else: iprot.skip(ftype) elif fid == 7: if ftype == TType.STRUCT: self.stats = ExecutorStats() self.stats.read(iprot) else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('ExecutorSummary') if self.executor_info is not None: oprot.writeFieldBegin('executor_info', TType.STRUCT, 1) self.executor_info.write(oprot) oprot.writeFieldEnd() if self.component_id is not None: oprot.writeFieldBegin('component_id', TType.STRING, 2) oprot.writeString(self.component_id.encode('utf-8')) oprot.writeFieldEnd() if self.host is not None: oprot.writeFieldBegin('host', TType.STRING, 3) oprot.writeString(self.host.encode('utf-8')) oprot.writeFieldEnd() if self.port is not None: oprot.writeFieldBegin('port', TType.I32, 4) oprot.writeI32(self.port) oprot.writeFieldEnd() if self.uptime_secs is not None: oprot.writeFieldBegin('uptime_secs', TType.I32, 5) oprot.writeI32(self.uptime_secs) oprot.writeFieldEnd() if self.stats is not None: oprot.writeFieldBegin('stats', TType.STRUCT, 7) self.stats.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.executor_info is None: raise TProtocol.TProtocolException(message='Required field executor_info is unset!') if self.component_id is None: raise TProtocol.TProtocolException(message='Required field component_id is unset!') if self.host is None: raise TProtocol.TProtocolException(message='Required field host is unset!') if self.port is None: raise TProtocol.TProtocolException(message='Required field port is unset!') if self.uptime_secs is None: raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class TopologyInfo: """ Attributes: - id - name - uptime_secs - executors - status - errors """ thrift_spec = ( None, # 0 (1, TType.STRING, 'id', None, None, ), # 1 (2, TType.STRING, 'name', None, None, ), # 2 (3, TType.I32, 'uptime_secs', None, None, ), # 3 (4, TType.LIST, 'executors', (TType.STRUCT,(ExecutorSummary, ExecutorSummary.thrift_spec)), None, ), # 4 (5, TType.STRING, 'status', None, None, ), # 5 (6, TType.MAP, 'errors', (TType.STRING,None,TType.LIST,(TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec))), None, ), # 6 ) def __hash__(self): return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors) def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None,): self.id = id self.name = name self.uptime_secs = uptime_secs self.executors = executors self.status = status self.errors = errors def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.name = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 3: if ftype == TType.I32: self.uptime_secs = iprot.readI32(); else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.executors = [] (_etype263, _size260) = iprot.readListBegin() for _i264 in xrange(_size260): _elem265 = ExecutorSummary() _elem265.read(iprot) self.executors.append(_elem265) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.STRING: self.status = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 6: if ftype == TType.MAP: self.errors = {} (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin() for _i270 in xrange(_size266): _key271 = iprot.readString().decode('utf-8') _val272 = [] (_etype276, _size273) = iprot.readListBegin() for _i277 in xrange(_size273): _elem278 = ErrorInfo() _elem278.read(iprot) _val272.append(_elem278) iprot.readListEnd() self.errors[_key271] = _val272 iprot.readMapEnd() else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('TopologyInfo') if self.id is not None: oprot.writeFieldBegin('id', TType.STRING, 1) oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 2) oprot.writeString(self.name.encode('utf-8')) oprot.writeFieldEnd() if self.uptime_secs is not None: oprot.writeFieldBegin('uptime_secs', TType.I32, 3) oprot.writeI32(self.uptime_secs) oprot.writeFieldEnd() if self.executors is not None: oprot.writeFieldBegin('executors', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.executors)) for iter279 in self.executors: iter279.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.status is not None: oprot.writeFieldBegin('status', TType.STRING, 5) oprot.writeString(self.status.encode('utf-8')) oprot.writeFieldEnd() if self.errors is not None: oprot.writeFieldBegin('errors', TType.MAP, 6) oprot.writeMapBegin(TType.STRING, TType.LIST, len(self.errors)) for kiter280,viter281 in self.errors.items(): oprot.writeString(kiter280.encode('utf-8')) oprot.writeListBegin(TType.STRUCT, len(viter281)) for iter282 in viter281: iter282.write(oprot) oprot.writeListEnd() oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.id is None: raise TProtocol.TProtocolException(message='Required field id is unset!') if self.name is None: raise TProtocol.TProtocolException(message='Required field name is unset!') if self.uptime_secs is None: raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') if self.executors is None: raise TProtocol.TProtocolException(message='Required field executors is unset!') if self.status is None: raise TProtocol.TProtocolException(message='Required field status is unset!') if self.errors is None: raise TProtocol.TProtocolException(message='Required field errors is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class KillOptions: """ Attributes: - wait_secs """ thrift_spec = ( None, # 0 (1, TType.I32, 'wait_secs', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.wait_secs) def __init__(self, wait_secs=None,): self.wait_secs = wait_secs def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.I32: self.wait_secs = iprot.readI32(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('KillOptions') if self.wait_secs is not None: oprot.writeFieldBegin('wait_secs', TType.I32, 1) oprot.writeI32(self.wait_secs) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class RebalanceOptions: """ Attributes: - wait_secs - num_workers - num_executors """ thrift_spec = ( None, # 0 (1, TType.I32, 'wait_secs', None, None, ), # 1 (2, TType.I32, 'num_workers', None, None, ), # 2 (3, TType.MAP, 'num_executors', (TType.STRING,None,TType.I32,None), None, ), # 3 ) def __hash__(self): return 0 + hash(self.wait_secs) + hash(self.num_workers) + hash(self.num_executors) def __init__(self, wait_secs=None, num_workers=None, num_executors=None,): self.wait_secs = wait_secs self.num_workers = num_workers self.num_executors = num_executors def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.I32: self.wait_secs = iprot.readI32(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.I32: self.num_workers = iprot.readI32(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.MAP: self.num_executors = {} (_ktype284, _vtype285, _size283 ) = iprot.readMapBegin() for _i287 in xrange(_size283): _key288 = iprot.readString().decode('utf-8') _val289 = iprot.readI32(); self.num_executors[_key288] = _val289 iprot.readMapEnd() else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('RebalanceOptions') if self.wait_secs is not None: oprot.writeFieldBegin('wait_secs', TType.I32, 1) oprot.writeI32(self.wait_secs) oprot.writeFieldEnd() if self.num_workers is not None: oprot.writeFieldBegin('num_workers', TType.I32, 2) oprot.writeI32(self.num_workers) oprot.writeFieldEnd() if self.num_executors is not None: oprot.writeFieldBegin('num_executors', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.I32, len(self.num_executors)) for kiter290,viter291 in self.num_executors.items(): oprot.writeString(kiter290.encode('utf-8')) oprot.writeI32(viter291) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class SubmitOptions: """ Attributes: - initial_status """ thrift_spec = ( None, # 0 (1, TType.I32, 'initial_status', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.initial_status) def __init__(self, initial_status=None,): self.initial_status = initial_status def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.I32: self.initial_status = iprot.readI32(); else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('SubmitOptions') if self.initial_status is not None: oprot.writeFieldBegin('initial_status', TType.I32, 1) oprot.writeI32(self.initial_status) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.initial_status is None: raise TProtocol.TProtocolException(message='Required field initial_status is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class DRPCRequest: """ Attributes: - func_args - request_id """ thrift_spec = ( None, # 0 (1, TType.STRING, 'func_args', None, None, ), # 1 (2, TType.STRING, 'request_id', None, None, ), # 2 ) def __hash__(self): return 0 + hash(self.func_args) + hash(self.request_id) def __init__(self, func_args=None, request_id=None,): self.func_args = func_args self.request_id = request_id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.func_args = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: self.request_id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('DRPCRequest') if self.func_args is not None: oprot.writeFieldBegin('func_args', TType.STRING, 1) oprot.writeString(self.func_args.encode('utf-8')) oprot.writeFieldEnd() if self.request_id is not None: oprot.writeFieldBegin('request_id', TType.STRING, 2) oprot.writeString(self.request_id.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.func_args is None: raise TProtocol.TProtocolException(message='Required field func_args is unset!') if self.request_id is None: raise TProtocol.TProtocolException(message='Required field request_id is unset!') return def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) class DRPCExecutionException(Exception): """ Attributes: - msg """ thrift_spec = ( None, # 0 (1, TType.STRING, 'msg', None, None, ), # 1 ) def __hash__(self): return 0 + hash(self.msg) def __init__(self, msg=None,): self.msg = msg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) return iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break if fid == 1: if ftype == TType.STRING: self.msg = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() iprot.readStructEnd() def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('DRPCExecutionException') if self.msg is not None: oprot.writeFieldBegin('msg', TType.STRING, 1) oprot.writeString(self.msg.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): if self.msg is None: raise TProtocol.TProtocolException(message='Required field msg is unset!') return def __str__(self): return repr(self) def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) def __eq__(self, other): return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not (self == other) ================================================ FILE: storm-core/src/storm.thrift ================================================ #!/usr/local/bin/thrift --gen java:beans,nocamel,hashcode /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, * software distributed under the License is distributed on an * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. * * Contains some contributions under the Thrift Software License. * Please see doc/old-thrift-license.txt in the Thrift distribution for * details. */ namespace java backtype.storm.generated union JavaObjectArg { 1: i32 int_arg; 2: i64 long_arg; 3: string string_arg; 4: bool bool_arg; 5: binary binary_arg; 6: double double_arg; } struct JavaObject { 1: required string full_class_name; 2: required list args_list; } struct NullStruct { } struct GlobalStreamId { 1: required string componentId; 2: required string streamId; #Going to need to add an enum for the stream type (NORMAL or FAILURE) } union Grouping { 1: list fields; //empty list means global grouping 2: NullStruct shuffle; // tuple is sent to random task 3: NullStruct all; // tuple is sent to every task 4: NullStruct none; // tuple is sent to a single task (storm's choice) -> allows storm to optimize the topology by bundling tasks into a single process 5: NullStruct direct; // this bolt expects the source bolt to send tuples directly to it 6: JavaObject custom_object; 7: binary custom_serialized; 8: NullStruct local_or_shuffle; // prefer sending to tasks in the same worker process, otherwise shuffle } struct StreamInfo { 1: required list output_fields; 2: required bool direct; } struct ShellComponent { // should change this to 1: required list execution_command; 1: string execution_command; 2: string script; } union ComponentObject { 1: binary serialized_java; 2: ShellComponent shell; 3: JavaObject java_object; } struct ComponentCommon { 1: required map inputs; 2: required map streams; //key is stream id 3: optional i32 parallelism_hint; //how many threads across the cluster should be dedicated to this component // component specific configuration respects: // topology.debug: false // topology.max.task.parallelism: null // can replace isDistributed with this // topology.max.spout.pending: null // topology.kryo.register // this is the only additive one // component specific configuration 4: optional string json_conf; } struct SpoutSpec { 1: required ComponentObject spout_object; 2: required ComponentCommon common; // can force a spout to be non-distributed by overriding the component configuration // and setting TOPOLOGY_MAX_TASK_PARALLELISM to 1 } struct Bolt { 1: required ComponentObject bolt_object; 2: required ComponentCommon common; } // not implemented yet // this will eventually be the basis for subscription implementation in storm struct StateSpoutSpec { 1: required ComponentObject state_spout_object; 2: required ComponentCommon common; } struct StormTopology { //ids must be unique across maps // #workers to use is in conf 1: required map spouts; 2: required map bolts; 3: required map state_spouts; } exception AlreadyAliveException { 1: required string msg; } exception NotAliveException { 1: required string msg; } exception InvalidTopologyException { 1: required string msg; } struct TopologySummary { 1: required string id; 2: required string name; 3: required i32 num_tasks; 4: required i32 num_executors; 5: required i32 num_workers; 6: required i32 uptime_secs; 7: required string status; } struct SupervisorSummary { 1: required string host; 2: required i32 uptime_secs; 3: required i32 num_workers; 4: required i32 num_used_workers; 5: required string supervisor_id; } struct ClusterSummary { 1: required list supervisors; 2: required i32 nimbus_uptime_secs; 3: required list topologies; } struct ErrorInfo { 1: required string error; 2: required i32 error_time_secs; } struct BoltStats { 1: required map> acked; 2: required map> failed; 3: required map> process_ms_avg; 4: required map> executed; 5: required map> execute_ms_avg; } struct SpoutStats { 1: required map> acked; 2: required map> failed; 3: required map> complete_ms_avg; } union ExecutorSpecificStats { 1: BoltStats bolt; 2: SpoutStats spout; } // Stats are a map from the time window (all time or a number indicating number of seconds in the window) // to the stats. Usually stats are a stream id to a count or average. struct ExecutorStats { 1: required map> emitted; 2: required map> transferred; 3: required ExecutorSpecificStats specific; } struct ExecutorInfo { 1: required i32 task_start; 2: required i32 task_end; } struct ExecutorSummary { 1: required ExecutorInfo executor_info; 2: required string component_id; 3: required string host; 4: required i32 port; 5: required i32 uptime_secs; 7: optional ExecutorStats stats; } struct TopologyInfo { 1: required string id; 2: required string name; 3: required i32 uptime_secs; 4: required list executors; 5: required string status; 6: required map> errors; } struct KillOptions { 1: optional i32 wait_secs; } struct RebalanceOptions { 1: optional i32 wait_secs; 2: optional i32 num_workers; 3: optional map num_executors; } enum TopologyInitialStatus { ACTIVE = 1, INACTIVE = 2 } struct SubmitOptions { 1: required TopologyInitialStatus initial_status; } service Nimbus { void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); void killTopology(1: string name) throws (1: NotAliveException e); void killTopologyWithOpts(1: string name, 2: KillOptions options) throws (1: NotAliveException e); void activate(1: string name) throws (1: NotAliveException e); void deactivate(1: string name) throws (1: NotAliveException e); void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite); // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs string beginFileUpload(); void uploadChunk(1: string location, 2: binary chunk); void finishFileUpload(1: string location); string beginFileDownload(1: string file); //can stop downloading chunks when receive 0-length byte array back binary downloadChunk(1: string id); // returns json string getNimbusConf(); // stats functions ClusterSummary getClusterInfo(); TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e); //returns json string getTopologyConf(1: string id) throws (1: NotAliveException e); StormTopology getTopology(1: string id) throws (1: NotAliveException e); StormTopology getUserTopology(1: string id) throws (1: NotAliveException e); } struct DRPCRequest { 1: required string func_args; 2: required string request_id; } exception DRPCExecutionException { 1: required string msg; } service DistributedRPC { string execute(1: string functionName, 2: string funcArgs) throws (1: DRPCExecutionException e); } service DistributedRPCInvocations { void result(1: string id, 2: string result); DRPCRequest fetchRequest(1: string functionName); void failRequest(1: string id); } ================================================ FILE: storm-core/src/ui/public/css/bootstrap-1.1.0.css ================================================ /*! * Bootstrap v1.1.0 * * Copyright 2011 Twitter, Inc * Licensed under the Apache License v2.0 * http://www.apache.org/licenses/LICENSE-2.0 * * Designed and built with all the love in the world @twitter by @mdo and @fat. * Date: Mon Aug 22 23:50:35 PDT 2011 */ /* Reset.less * Props to Eric Meyer (meyerweb.com) for his CSS reset file. We're using an adapted version here that cuts out some of the reset HTML elements we will never need here (i.e., dfn, samp, etc). * ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- */ html, body { margin: 0; padding: 0; } h1, h2, h3, h4, h5, h6, p, blockquote, pre, a, abbr, acronym, address, cite, code, del, dfn, em, img, q, s, samp, small, strike, strong, sub, sup, tt, var, dd, dl, dt, li, ol, ul, fieldset, form, label, legend, button, table, caption, tbody, tfoot, thead, tr, th, td { margin: 0; padding: 0; border: 0; font-weight: normal; font-style: normal; font-size: 100%; line-height: 1; font-family: inherit; } table { border-collapse: collapse; border-spacing: 0; } ol, ul { list-style: none; } q:before, q:after, blockquote:before, blockquote:after { content: ""; } header, section, footer, article, aside { display: block; } /* Preboot.less * Variables and mixins to pre-ignite any new web development project * ------------------------------------------------------------------ */ .clearfix { zoom: 1; } .clearfix:before, .clearfix:after { display: table; content: ""; } .clearfix:after { clear: both; } .center-block { display: block; margin: 0 auto; } .container { width: 940px; margin: 0 auto; zoom: 1; } .container:before, .container:after { display: table; content: ""; } .container:after { clear: both; } /* * Scaffolding * Basic and global styles for generating a grid system, structural layout, and page templates * ------------------------------------------------------------------------------------------- */ .row { zoom: 1; } .row:before, .row:after { display: table; content: ""; } .row:after { clear: both; } .row .span1 { float: left; width: 40px; margin-left: 20px; } .row .span1:first-child { margin-left: 0; } .row .span2 { float: left; width: 100px; margin-left: 20px; } .row .span2:first-child { margin-left: 0; } .row .span3 { float: left; width: 160px; margin-left: 20px; } .row .span3:first-child { margin-left: 0; } .row .span4 { float: left; width: 220px; margin-left: 20px; } .row .span4:first-child { margin-left: 0; } .row .span5 { float: left; width: 280px; margin-left: 20px; } .row .span5:first-child { margin-left: 0; } .row .span6 { float: left; width: 340px; margin-left: 20px; } .row .span6:first-child { margin-left: 0; } .row .span7 { float: left; width: 400px; margin-left: 20px; } .row .span7:first-child { margin-left: 0; } .row .span8 { float: left; width: 460px; margin-left: 20px; } .row .span8:first-child { margin-left: 0; } .row .span9 { float: left; width: 520px; margin-left: 20px; } .row .span9:first-child { margin-left: 0; } .row .span10 { float: left; width: 580px; margin-left: 20px; } .row .span10:first-child { margin-left: 0; } .row .span11 { float: left; width: 640px; margin-left: 20px; } .row .span11:first-child { margin-left: 0; } .row .span12 { float: left; width: 700px; margin-left: 20px; } .row .span12:first-child { margin-left: 0; } .row .span13 { float: left; width: 760px; margin-left: 20px; } .row .span13:first-child { margin-left: 0; } .row .span14 { float: left; width: 820px; margin-left: 20px; } .row .span14:first-child { margin-left: 0; } .row .span15 { float: left; width: 880px; margin-left: 20px; } .row .span15:first-child { margin-left: 0; } .row .span16 { float: left; width: 940px; margin-left: 20px; } .row .span16:first-child { margin-left: 0; } .row .offset1 { margin-left: 80px !important; } .row .offset1:first-child { margin-left: 60px !important; } .row .offset2 { margin-left: 140px !important; } .row .offset2:first-child { margin-left: 120px !important; } .row .offset3 { margin-left: 200px !important; } .row .offset3:first-child { margin-left: 180px !important; } .row .offset4 { margin-left: 260px !important; } .row .offset4:first-child { margin-left: 240px !important; } .row .offset5 { margin-left: 320px !important; } .row .offset5:first-child { margin-left: 300px !important; } .row .offset6 { margin-left: 380px !important; } .row .offset6:first-child { margin-left: 360px !important; } .row .offset7 { margin-left: 440px !important; } .row .offset7:first-child { margin-left: 420px !important; } .row .offset8 { margin-left: 500px !important; } .row .offset8:first-child { margin-left: 480px !important; } .row .offset9 { margin-left: 500px !important; } .row .offset9:first-child { margin-left: 480px !important; } .row .offset10 { margin-left: 620px !important; } .row .offset10:first-child { margin-left: 600px !important; } .row .offset11 { margin-left: 680px !important; } .row .offset11:first-child { margin-left: 660px !important; } .row .offset12 { margin-left: 740px !important; } .row .offset12:first-child { margin-left: 720px !important; } html, body { background-color: #fff; } body { margin: 0; font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; font-size: 13px; font-weight: normal; line-height: 18px; color: #808080; text-rendering: optimizeLegibility; } div.container { width: 940px; margin: 0 auto; } div.container-fluid { padding: 0 20px; zoom: 1; } div.container-fluid:before, div.container-fluid:after { display: table; content: ""; } div.container-fluid:after { clear: both; } div.container-fluid div.sidebar { float: left; width: 220px; } div.container-fluid div.content { min-width: 700px; max-width: 1180px; margin-left: 240px; } a { color: #0069d6; text-decoration: none; line-height: inherit; font-weight: inherit; } a:hover { color: #0050a3; text-decoration: underline; } .btn { cursor: pointer; display: inline-block; background-color: #e6e6e6; background-repeat: no-repeat; background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), color-stop(0.25, #ffffff), to(#e6e6e6)); background-image: -webkit-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6); background-image: -moz-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6); background-image: -ms-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6); background-image: -o-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6); background-image: linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6); padding: 4px 14px; text-shadow: 0 1px 1px rgba(255, 255, 255, 0.75); color: #333; font-size: 13px; line-height: 18px; border: 1px solid #ccc; border-bottom-color: #bbb; -webkit-border-radius: 4px; -moz-border-radius: 4px; border-radius: 4px; -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); -moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); -webkit-transition: 0.1s linear all; -moz-transition: 0.1s linear all; transition: 0.1s linear all; } .btn:hover { background-position: 0 -15px; color: #333; text-decoration: none; } .btn.primary, .btn.danger { color: #fff; } .btn.primary:hover, .btn.danger:hover { color: #fff; } .btn.primary { background-color: #0064cd; background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(#049cdb), to(#0064cd)); background-image: -moz-linear-gradient(#049cdb, #0064cd); background-image: -ms-linear-gradient(#049cdb, #0064cd); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #049cdb), color-stop(100%, #0064cd)); background-image: -webkit-linear-gradient(#049cdb, #0064cd); background-image: -o-linear-gradient(#049cdb, #0064cd); background-image: linear-gradient(#049cdb, #0064cd); text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); border-color: #0064cd #0064cd #003f81; border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); } .btn.danger { background-color: #9d261d; background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(#d83a2e), to(#9d261d)); background-image: -moz-linear-gradient(#d83a2e, #9d261d); background-image: -ms-linear-gradient(#d83a2e, #9d261d); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #d83a2e), color-stop(100%, #9d261d)); background-image: -webkit-linear-gradient(#d83a2e, #9d261d); background-image: -o-linear-gradient(#d83a2e, #9d261d); background-image: linear-gradient(#d83a2e, #9d261d); text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); border-color: #9d261d #9d261d #5c1611; border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); } .btn.large { font-size: 16px; line-height: 28px; -webkit-border-radius: 6px; -moz-border-radius: 6px; border-radius: 6px; } .btn.small { padding-right: 9px; padding-left: 9px; font-size: 11px; } .btn.disabled { background-image: none; filter: alpha(opacity=65); -khtml-opacity: 0.65; -moz-opacity: 0.65; opacity: 0.65; cursor: default; } .btn:disabled { background-image: none; filter: alpha(opacity=65); -khtml-opacity: 0.65; -moz-opacity: 0.65; opacity: 0.65; cursor: default; } .btn:active { -webkit-box-shadow: inset 0 3px 7px rgba(0, 0, 0, 0.1), 0 1px 2px rgba(0, 0, 0, 0.05); -moz-box-shadow: inset 0 3px 7px rgba(0, 0, 0, 0.1), 0 1px 2px rgba(0, 0, 0, 0.05); box-shadow: inset 0 3px 7px rgba(0, 0, 0, 0.1), 0 1px 2px rgba(0, 0, 0, 0.05); } button.btn::-moz-focus-inner, input[type=submit].btn::-moz-focus-inner { padding: 0; border: 0; } /* Typography.less * Headings, body text, lists, code, and more for a versatile and durable typography system * ---------------------------------------------------------------------------------------- */ p { font-size: 13px; font-weight: normal; line-height: 18px; margin-bottom: 9px; } p small { font-size: 11px; color: #bfbfbf; } h1, h2, h3, h4, h5, h6 { font-weight: bold; color: #404040; } h1 small, h2 small, h3 small, h4 small, h5 small, h6 small { color: #bfbfbf; } h1 { margin-bottom: 18px; font-size: 30px; line-height: 36px; } h1 small { font-size: 18px; } h2 { font-size: 24px; line-height: 36px; } h2 small { font-size: 14px; } h3, h4, h5, h6 { line-height: 36px; } h3 { font-size: 18px; } h3 small { font-size: 14px; } h4 { font-size: 16px; } h4 small { font-size: 12px; } h5 { font-size: 14px; } h6 { font-size: 13px; color: #bfbfbf; text-transform: uppercase; } ul, ol { margin: 0 0 18px 25px; } ul ul, ul ol, ol ol, ol ul { margin-bottom: 0; } ul { list-style: disc; } ol { list-style: decimal; } li { line-height: 18px; color: #808080; } ul.unstyled { list-style: none; margin-left: 0; } dl { margin-bottom: 18px; } dl dt, dl dd { line-height: 18px; } dl dt { font-weight: bold; } dl dd { margin-left: 9px; } hr { margin: 0 0 19px; border: 0; border-bottom: 1px solid #eee; } strong { font-style: inherit; font-weight: bold; line-height: inherit; } em { font-style: italic; font-weight: inherit; line-height: inherit; } .muted { color: #e6e6e6; } blockquote { margin-bottom: 18px; border-left: 5px solid #eee; padding-left: 15px; } blockquote p { font-size: 14px; font-weight: 300; line-height: 18px; margin-bottom: 0; } blockquote small { display: block; font-size: 12px; font-weight: 300; line-height: 18px; color: #bfbfbf; } blockquote small:before { content: '\2014 \00A0'; } address { display: block; line-height: 18px; margin-bottom: 18px; } code, pre { padding: 0 3px 2px; font-family: Monaco, Andale Mono, Courier New, monospace; font-size: 12px; -webkit-border-radius: 3px; -moz-border-radius: 3px; border-radius: 3px; } code { background-color: #fee9cc; color: rgba(0, 0, 0, 0.75); padding: 1px 3px; } pre { background-color: #f5f5f5; display: block; padding: 17px; margin: 0 0 18px; line-height: 18px; font-size: 12px; border: 1px solid #ccc; border: 1px solid rgba(0, 0, 0, 0.15); -webkit-border-radius: 3px; -moz-border-radius: 3px; border-radius: 3px; white-space: pre-wrap; } /* Forms.less * Base styles for various input types, form layouts, and states * ------------------------------------------------------------- */ form { margin-bottom: 18px; } form fieldset { margin-bottom: 18px; padding-top: 18px; } form fieldset legend { display: block; margin-left: 150px; font-size: 20px; line-height: 1; *margin: 0 0 5px 145px; /* IE6-7 */ *line-height: 1.5; /* IE6-7 */ color: #404040; } form .clearfix { margin-bottom: 18px; } form label, form input, form select, form textarea { font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; font-size: 13px; font-weight: normal; line-height: normal; } form label { padding-top: 6px; font-size: 13px; line-height: 18px; float: left; width: 130px; text-align: right; color: #404040; } form div.input { margin-left: 150px; } form input[type=checkbox], form input[type=radio] { cursor: pointer; } form input[type=text], form input[type=password], form textarea, form select, form .uneditable-input { display: inline-block; width: 210px; margin: 0; padding: 4px; font-size: 13px; line-height: 18px; height: 18px; color: #808080; border: 1px solid #ccc; -webkit-border-radius: 3px; -moz-border-radius: 3px; border-radius: 3px; } form select, form input[type=file] { height: 27px; line-height: 27px; } form textarea { height: auto; } form .uneditable-input { background-color: #eee; display: block; border-color: #ccc; -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.075); -moz-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.075); box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.075); } form :-moz-placeholder { color: #bfbfbf; } form ::-webkit-input-placeholder { color: #bfbfbf; } form input[type=text], form input[type=password], form select, form textarea { -webkit-transition: border linear 0.2s, box-shadow linear 0.2s; -moz-transition: border linear 0.2s, box-shadow linear 0.2s; transition: border linear 0.2s, box-shadow linear 0.2s; -webkit-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1); -moz-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1); box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1); } form input[type=text]:focus, form input[type=password]:focus, form textarea:focus { outline: none; border-color: rgba(82, 168, 236, 0.8); -webkit-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1), 0 0 8px rgba(82, 168, 236, 0.6); -moz-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1), 0 0 8px rgba(82, 168, 236, 0.6); box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1), 0 0 8px rgba(82, 168, 236, 0.6); } form div.error { background: #fae5e3; padding: 10px 0; margin: -10px 0 10px; -webkit-border-radius: 4px; -moz-border-radius: 4px; border-radius: 4px; } form div.error > label, form div.error span.help-inline, form div.error span.help-block { color: #9d261d; } form div.error input[type=text], form div.error input[type=password], form div.error textarea { border-color: #c87872; -webkit-box-shadow: 0 0 3px rgba(171, 41, 32, 0.25); -moz-box-shadow: 0 0 3px rgba(171, 41, 32, 0.25); box-shadow: 0 0 3px rgba(171, 41, 32, 0.25); } form div.error input[type=text]:focus, form div.error input[type=password]:focus, form div.error textarea:focus { border-color: #b9554d; -webkit-box-shadow: 0 0 6px rgba(171, 41, 32, 0.5); -moz-box-shadow: 0 0 6px rgba(171, 41, 32, 0.5); box-shadow: 0 0 6px rgba(171, 41, 32, 0.5); } form div.error .input-prepend span.add-on, form div.error .input-append span.add-on { background: #f4c8c5; border-color: #c87872; color: #b9554d; } form .input-mini, form input.mini, form textarea.mini, form select.mini { width: 60px; } form .input-small, form input.small, form textarea.small, form select.small { width: 90px; } form .input-medium, form input.medium, form textarea.medium, form select.medium { width: 150px; } form .input-large, form input.large, form textarea.large, form select.large { width: 210px; } form .input-xlarge, form input.xlarge, form textarea.xlarge, form select.xlarge { width: 270px; } form .input-xxlarge, form input.xxlarge, form textarea.xxlarge, form select.xxlarge { width: 530px; } form textarea.xxlarge { overflow-y: scroll; } form input[readonly]:focus, form textarea[readonly]:focus, form input.disabled { background: #f5f5f5; border-color: #ddd; -webkit-box-shadow: none; -moz-box-shadow: none; box-shadow: none; } .actions { background: #f5f5f5; margin-top: 18px; margin-bottom: 18px; padding: 17px 20px 18px 150px; border-top: 1px solid #ddd; -webkit-border-radius: 0 0 3px 3px; -moz-border-radius: 0 0 3px 3px; border-radius: 0 0 3px 3px; } .actions .secondary-action { float: right; } .actions .secondary-action a { line-height: 30px; } .actions .secondary-action a:hover { text-decoration: underline; } .help-inline, .help-block { font-size: 12px; line-height: 18px; color: #bfbfbf; } .help-inline { padding-left: 5px; *position: relative; /* IE6-7 */ *top: -5px; /* IE6-7 */ } .help-block { display: block; max-width: 600px; } .inline-inputs { color: #808080; } .inline-inputs span, .inline-inputs input[type=text] { display: inline-block; } .inline-inputs input.mini { width: 60px; } .inline-inputs input.small { width: 90px; } .inline-inputs span { padding: 0 2px 0 1px; } .input-prepend input[type=text], .input-append input[type=text], .input-prepend input[type=password], .input-append input[type=password] { -webkit-border-radius: 0 3px 3px 0; -moz-border-radius: 0 3px 3px 0; border-radius: 0 3px 3px 0; } .input-prepend .add-on, .input-append .add-on { background: #f5f5f5; float: left; display: block; width: auto; min-width: 16px; padding: 4px 4px 4px 5px; color: #bfbfbf; font-weight: normal; line-height: 18px; height: 18px; text-align: center; text-shadow: 0 1px 0 #fff; border: 1px solid #ccc; border-right-width: 0; -webkit-border-radius: 3px 0 0 3px; -moz-border-radius: 3px 0 0 3px; border-radius: 3px 0 0 3px; } .input-prepend .active, .input-append .active { background: #a9dba9; border-color: #46a546; } .input-prepend .add-on { *margin-top: 1px; /* IE6-7 */ } .input-append input[type=text], .input-append input[type=password] { float: left; -webkit-border-radius: 3px 0 0 3px; -moz-border-radius: 3px 0 0 3px; border-radius: 3px 0 0 3px; } .input-append .add-on { -webkit-border-radius: 0 3px 3px 0; -moz-border-radius: 0 3px 3px 0; border-radius: 0 3px 3px 0; border-right-width: 1px; border-left-width: 0; } .inputs-list { margin: 0 0 5px; width: 100%; } .inputs-list li { display: block; padding: 0; width: 100%; } .inputs-list li label { display: block; float: none; width: auto; padding: 0; line-height: 18px; text-align: left; white-space: normal; } .inputs-list li label strong { color: #808080; } .inputs-list li label small { font-size: 12px; font-weight: normal; } .inputs-list li ul.inputs-list { margin-left: 25px; margin-bottom: 10px; padding-top: 0; } .inputs-list li:first-child { padding-top: 5px; } .inputs-list input[type=radio], .inputs-list input[type=checkbox] { margin-bottom: 0; } form.form-stacked { padding-left: 20px; } form.form-stacked fieldset { padding-top: 9px; } form.form-stacked legend { margin-left: 0; } form.form-stacked label { display: block; float: none; width: auto; font-weight: bold; text-align: left; line-height: 20px; padding-top: 0; } form.form-stacked .clearfix { margin-bottom: 9px; } form.form-stacked .clearfix div.input { margin-left: 0; } form.form-stacked .inputs-list { margin-bottom: 0; } form.form-stacked .inputs-list li { padding-top: 0; } form.form-stacked .inputs-list li label { font-weight: normal; padding-top: 0; } form.form-stacked div.error { padding-top: 10px; padding-bottom: 10px; padding-left: 10px; margin-top: 0; margin-left: -10px; } form.form-stacked .actions { margin-left: -20px; padding-left: 20px; } /* * Tables.less * Tables for, you guessed it, tabular data * ---------------------------------------- */ table { width: 100%; margin-bottom: 18px; padding: 0; border-collapse: separate; font-size: 13px; } table th, table td { padding: 10px 10px 9px; line-height: 13.5px; text-align: left; vertical-align: middle; border-bottom: 1px solid #ddd; } table th { padding-top: 9px; font-weight: bold; border-bottom-width: 2px; } .zebra-striped tbody tr:nth-child(odd) td { background-color: #f9f9f9; } .zebra-striped tbody tr:hover td { background-color: #f5f5f5; } .zebra-striped .header { cursor: pointer; } .zebra-striped .header:after { content: ""; float: right; margin-top: 7px; border-width: 0 4px 4px; border-style: solid; border-color: #000 transparent; visibility: hidden; } .zebra-striped .headerSortUp, .zebra-striped .headerSortDown { background-color: rgba(141, 192, 219, 0.25); text-shadow: 0 1px 1px rgba(255, 255, 255, 0.75); -webkit-border-radius: 3px 3px 0 0; -moz-border-radius: 3px 3px 0 0; border-radius: 3px 3px 0 0; } .zebra-striped .header:hover:after { visibility: visible; } .zebra-striped .headerSortDown:after, .zebra-striped .headerSortDown:hover:after { visibility: visible; filter: alpha(opacity=60); -khtml-opacity: 0.6; -moz-opacity: 0.6; opacity: 0.6; } .zebra-striped .headerSortUp:after { border-bottom: none; border-left: 4px solid transparent; border-right: 4px solid transparent; border-top: 4px solid #000; visibility: visible; -webkit-box-shadow: none; -moz-box-shadow: none; box-shadow: none; filter: alpha(opacity=60); -khtml-opacity: 0.6; -moz-opacity: 0.6; opacity: 0.6; } table .blue { color: #049cdb; border-bottom-color: #049cdb; } table .headerSortUp.blue, table .headerSortDown.blue { background-color: #ade6fe; } table .green { color: #46a546; border-bottom-color: #46a546; } table .headerSortUp.green, table .headerSortDown.green { background-color: #cdeacd; } table .red { color: #9d261d; border-bottom-color: #9d261d; } table .headerSortUp.red, table .headerSortDown.red { background-color: #f4c8c5; } table .yellow { color: #ffc40d; border-bottom-color: #ffc40d; } table .headerSortUp.yellow, table .headerSortDown.yellow { background-color: #fff6d9; } table .orange { color: #f89406; border-bottom-color: #f89406; } table .headerSortUp.orange, table .headerSortDown.orange { background-color: #fee9cc; } table .purple { color: #7a43b6; border-bottom-color: #7a43b6; } table .headerSortUp.purple, table .headerSortDown.purple { background-color: #e2d5f0; } /* Patterns.less * Repeatable UI elements outside the base styles provided from the scaffolding * ---------------------------------------------------------------------------- */ .topbar { height: 40px; position: fixed; top: 0; left: 0; right: 0; z-index: 10000; overflow: visible; } .topbar .fill { background: #222; background-color: #222222; background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(#333333), to(#222222)); background-image: -moz-linear-gradient(#333333, #222222); background-image: -ms-linear-gradient(#333333, #222222); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #333333), color-stop(100%, #222222)); background-image: -webkit-linear-gradient(#333333, #222222); background-image: -o-linear-gradient(#333333, #222222); background-image: linear-gradient(#333333, #222222); -webkit-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.25), inset 0 -1px 0 rgba(0, 0, 0, 0.1); -moz-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.25), inset 0 -1px 0 rgba(0, 0, 0, 0.1); box-shadow: 0 1px 3px rgba(0, 0, 0, 0.25), inset 0 -1px 0 rgba(0, 0, 0, 0.1); } .topbar a { color: #bfbfbf; text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); } .topbar a:hover, .topbar ul li.active a { background-color: #333; background-color: rgba(255, 255, 255, 0.05); color: #ffffff; text-decoration: none; } .topbar h3 { position: relative; } .topbar h3 a { float: left; display: block; padding: 8px 20px 12px; margin-left: -20px; color: #ffffff; font-size: 20px; font-weight: 200; line-height: 1; } .topbar form { float: left; margin: 5px 0 0 0; position: relative; filter: alpha(opacity=100); -khtml-opacity: 1; -moz-opacity: 1; opacity: 1; } .topbar form input { background-color: #444; background-color: rgba(255, 255, 255, 0.3); font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; font-size: normal; font-weight: 13px; line-height: 1; width: 220px; padding: 4px 9px; color: #fff; color: rgba(255, 255, 255, 0.75); border: 1px solid #111; -webkit-border-radius: 4px; -moz-border-radius: 4px; border-radius: 4px; -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0px rgba(255, 255, 255, 0.25); -moz-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0px rgba(255, 255, 255, 0.25); box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0px rgba(255, 255, 255, 0.25); -webkit-transition: none; -moz-transition: none; transition: none; } .topbar form input:-moz-placeholder { color: #e6e6e6; } .topbar form input::-webkit-input-placeholder { color: #e6e6e6; } .topbar form input:hover { background-color: #bfbfbf; background-color: rgba(255, 255, 255, 0.5); color: #fff; } .topbar form input:focus, .topbar form input.focused { outline: none; background-color: #fff; color: #404040; text-shadow: 0 1px 0 #fff; border: 0; padding: 5px 10px; -webkit-box-shadow: 0 0 3px rgba(0, 0, 0, 0.15); -moz-box-shadow: 0 0 3px rgba(0, 0, 0, 0.15); box-shadow: 0 0 3px rgba(0, 0, 0, 0.15); } .topbar ul { display: block; float: left; margin: 0 10px 0 0; position: relative; } .topbar ul.secondary-nav { float: right; margin-left: 10px; margin-right: 0; } .topbar ul li { display: block; float: left; font-size: 13px; } .topbar ul li a { display: block; float: none; padding: 10px 10px 11px; line-height: 19px; text-decoration: none; } .topbar ul li a:hover { color: #fff; text-decoration: none; } .topbar ul li.active a { background-color: #222; background-color: rgba(0, 0, 0, 0.5); } .topbar ul.primary-nav li ul { left: 0; } .topbar ul.secondary-nav li ul { right: 0; } .topbar ul li.menu { position: relative; } .topbar ul li.menu a.menu:after { width: 0px; height: 0px; display: inline-block; content: "↓"; text-indent: -99999px; vertical-align: top; margin-top: 8px; margin-left: 4px; border-left: 4px solid transparent; border-right: 4px solid transparent; border-top: 4px solid #fff; filter: alpha(opacity=50); -khtml-opacity: 0.5; -moz-opacity: 0.5; opacity: 0.5; } .topbar ul li.menu.open a.menu, .topbar ul li.menu.open a:hover { background-color: #444; background-color: rgba(255, 255, 255, 0.1); *background-color: #444; /* IE6-7 */ color: #fff; } .topbar ul li.menu.open ul { display: block; } .topbar ul li.menu.open ul li a { background-color: transparent; font-weight: normal; } .topbar ul li.menu.open ul li a:hover { background-color: rgba(255, 255, 255, 0.1); *background-color: #444; /* IE6-7 */ color: #fff; } .topbar ul li.menu.open ul li.active a { background-color: rgba(255, 255, 255, 0.1); font-weight: bold; } .topbar ul li ul { background-color: #333; float: left; display: none; position: absolute; top: 40px; min-width: 160px; max-width: 220px; _width: 160px; margin-left: 0; margin-right: 0; padding: 0; text-align: left; border: 0; zoom: 1; -webkit-border-radius: 0 0 5px 5px; -moz-border-radius: 0 0 5px 5px; border-radius: 0 0 5px 5px; -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.6); -moz-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.6); box-shadow: 0 1px 2px rgba(0, 0, 0, 0.6); } .topbar ul li ul li { float: none; clear: both; display: block; background: none; font-size: 12px; } .topbar ul li ul li a { display: block; padding: 6px 15px; clear: both; font-weight: normal; line-height: 19px; color: #bbb; } .topbar ul li ul li a:hover { background-color: #333; background-color: rgba(255, 255, 255, 0.25); color: #fff; } .topbar ul li ul li.divider { height: 1px; overflow: hidden; background: #222; background: rgba(0, 0, 0, 0.2); border-bottom: 1px solid rgba(255, 255, 255, 0.1); margin: 5px 0; } .topbar ul li ul li span { clear: both; display: block; background: rgba(0, 0, 0, 0.2); padding: 6px 15px; cursor: default; color: #808080; border-top: 1px solid rgba(0, 0, 0, 0.2); } .hero-unit { background-color: #f5f5f5; margin-top: 60px; margin-bottom: 30px; padding: 60px; -webkit-border-radius: 6px; -moz-border-radius: 6px; border-radius: 6px; } .hero-unit h1 { margin-bottom: 0; font-size: 60px; line-height: 1; letter-spacing: -1px; } .hero-unit p { font-size: 18px; font-weight: 200; line-height: 27px; } footer { margin-top: 17px; padding-top: 17px; border-top: 1px solid #eee; } .page-header { margin-bottom: 17px; border-bottom: 1px solid #ddd; -webkit-box-shadow: 0 1px 0 rgba(255, 255, 255, 0.5); -moz-box-shadow: 0 1px 0 rgba(255, 255, 255, 0.5); box-shadow: 0 1px 0 rgba(255, 255, 255, 0.5); } .page-header h1 { margin-bottom: 8px; } .alert-message { background-color: rgba(0, 0, 0, 0.15); background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(transparent), to(rgba(0, 0, 0, 0.15))); background-image: -moz-linear-gradient(transparent, rgba(0, 0, 0, 0.15)); background-image: -ms-linear-gradient(transparent, rgba(0, 0, 0, 0.15)); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, transparent), color-stop(100%, rgba(0, 0, 0, 0.15))); background-image: -webkit-linear-gradient(transparent, rgba(0, 0, 0, 0.15)); background-image: -o-linear-gradient(transparent, rgba(0, 0, 0, 0.15)); background-image: linear-gradient(transparent, rgba(0, 0, 0, 0.15)); filter: "progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#15000000')"; background-color: #e6e6e6; margin-bottom: 18px; padding: 8px 15px; color: #fff; text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.3); border-bottom: 1px solid rgba(0, 0, 0, 0.3); -webkit-border-radius: 4px; -moz-border-radius: 4px; border-radius: 4px; } .alert-message p { color: #fff; margin-bottom: 0; } .alert-message p + p { margin-top: 5px; } .alert-message.error { background-color: #d83a2e; background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(#e4776f), to(#d83a2e)); background-image: -moz-linear-gradient(#e4776f, #d83a2e); background-image: -ms-linear-gradient(#e4776f, #d83a2e); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #e4776f), color-stop(100%, #d83a2e)); background-image: -webkit-linear-gradient(#e4776f, #d83a2e); background-image: -o-linear-gradient(#e4776f, #d83a2e); background-image: linear-gradient(#e4776f, #d83a2e); border-bottom-color: #b32b21; } .alert-message.warning { background-color: #ffd040; background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(#ffe38d), to(#ffd040)); background-image: -moz-linear-gradient(#ffe38d, #ffd040); background-image: -ms-linear-gradient(#ffe38d, #ffd040); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #ffe38d), color-stop(100%, #ffd040)); background-image: -webkit-linear-gradient(#ffe38d, #ffd040); background-image: -o-linear-gradient(#ffe38d, #ffd040); background-image: linear-gradient(#ffe38d, #ffd040); border-bottom-color: #ffc40d; } .alert-message.success { background-color: #62bc62; background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(#97d397), to(#62bc62)); background-image: -moz-linear-gradient(#97d397, #62bc62); background-image: -ms-linear-gradient(#97d397, #62bc62); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #97d397), color-stop(100%, #62bc62)); background-image: -webkit-linear-gradient(#97d397, #62bc62); background-image: -o-linear-gradient(#97d397, #62bc62); background-image: linear-gradient(#97d397, #62bc62); border-bottom-color: #46a546; } .alert-message.info { background-color: #04aef4; background-repeat: repeat-x; background-image: -khtml-gradient(linear, left top, left bottom, from(#62cffc), to(#04aef4)); background-image: -moz-linear-gradient(#62cffc, #04aef4); background-image: -ms-linear-gradient(#62cffc, #04aef4); background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #62cffc), color-stop(100%, #04aef4)); background-image: -webkit-linear-gradient(#62cffc, #04aef4); background-image: -o-linear-gradient(#62cffc, #04aef4); background-image: linear-gradient(#62cffc, #04aef4); border-bottom-color: #049cdb; } .alert-message .close { float: right; margin-top: -2px; color: #000; font-size: 20px; font-weight: bold; text-shadow: 0 1px 0 #ffffff; filter: alpha(opacity=20); -khtml-opacity: 0.2; -moz-opacity: 0.2; opacity: 0.2; } .alert-message .close:hover { text-decoration: none; filter: alpha(opacity=40); -khtml-opacity: 0.4; -moz-opacity: 0.4; opacity: 0.4; } .block-message { margin-bottom: 18px; padding: 14px; color: #404040; color: rgba(0, 0, 0, 0.8); *color: #404040; /* IE 6-7 */ text-shadow: 0 1px 0 rgba(255, 255, 255, 0.25); -webkit-border-radius: 6px; -moz-border-radius: 6px; border-radius: 6px; } .block-message p { color: #404040; color: rgba(0, 0, 0, 0.8); *color: #404040; /* IE 6-7 */ margin-right: 30px; margin-bottom: 0; } .block-message ul { margin-bottom: 0; } .block-message strong { display: block; } .block-message.error { background: #f8dcda; border: 1px solid #f4c8c5; } .block-message.warning { background: #fff0c0; border: 1px solid #ffe38d; } .block-message.success { background: #dff1df; border: 1px solid #bbe2bb; } .block-message.info { background: #c7eefe; border: 1px solid #ade6fe; } .tabs, .pills { margin: 0 0 20px; padding: 0; zoom: 1; } .tabs:before, .pills:before, .tabs:after, .pills:after { display: table; content: ""; } .tabs:after, .pills:after { clear: both; } .tabs li, .pills li { display: inline; } .tabs li a, .pills li a { float: left; width: auto; } .tabs { width: 100%; border-bottom: 1px solid #bfbfbf; } .tabs li a { margin-bottom: -1px; margin-right: 2px; padding: 0 15px; line-height: 35px; -webkit-border-radius: 3px 3px 0 0; -moz-border-radius: 3px 3px 0 0; border-radius: 3px 3px 0 0; } .tabs li a:hover { background-color: #e6e6e6; border-bottom: 1px solid #bfbfbf; } .tabs li.active a { background-color: #fff; padding: 0 14px; border: 1px solid #ccc; border-bottom: 0; color: #808080; } .pills li a { margin: 5px 3px 5px 0; padding: 0 15px; text-shadow: 0 1px 1px #fff; line-height: 30px; -webkit-border-radius: 15px; -moz-border-radius: 15px; border-radius: 15px; } .pills li a:hover { background: #0050a3; color: #fff; text-decoration: none; text-shadow: 0 1px 1px rgba(0, 0, 0, 0.25); } .pills li.active a { background: #0069d6; color: #fff; text-shadow: 0 1px 1px rgba(0, 0, 0, 0.25); } .pagination { height: 36px; margin: 18px 0; } .pagination ul { float: left; margin: 0; border: 1px solid #ddd; border: 1px solid rgba(0, 0, 0, 0.15); -webkit-border-radius: 3px; -moz-border-radius: 3px; border-radius: 3px; -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05); -moz-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05); box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05); } .pagination ul li { display: inline; } .pagination ul li a { float: left; padding: 0 14px; line-height: 34px; border-right: 1px solid; border-right-color: #ddd; border-right-color: rgba(0, 0, 0, 0.15); *border-right-color: #ddd; /* IE6-7 */ text-decoration: none; } .pagination ul li a:hover, .pagination ul li.active a { background-color: #c7eefe; } .pagination ul li.disabled a, .pagination ul li.disabled a:hover { background-color: none; color: #bfbfbf; } .pagination ul li.next a { border: 0; } .well { background-color: #f5f5f5; margin-bottom: 20px; padding: 19px; min-height: 20px; border: 1px solid #eee; border: 1px solid rgba(0, 0, 0, 0.05); -webkit-border-radius: 4px; -moz-border-radius: 4px; border-radius: 4px; -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05); -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05); box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05); } .modal-backdrop { background-color: rgba(0, 0, 0, 0.5); position: fixed; top: 0; left: 0; right: 0; bottom: 0; z-index: 1000; } .modal { position: fixed; top: 50%; left: 50%; z-index: 2000; width: 560px; margin: -280px 0 0 -250px; background-color: #ffffff; border: 1px solid #999; border: 1px solid rgba(0, 0, 0, 0.3); *border: 1px solid #999; /* IE6-7 */ -webkit-border-radius: 6px; -moz-border-radius: 6px; border-radius: 6px; -webkit-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); -moz-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); -webkit-background-clip: padding-box; -moz-background-clip: padding-box; background-clip: padding-box; } .modal .modal-header { border-bottom: 1px solid #eee; padding: 5px 20px; } .modal .modal-header .close { position: absolute; right: 10px; top: 10px; color: #999; line-height: 10px; font-size: 18px; } .modal .modal-body { padding: 20px; } .modal .modal-footer { background-color: #f5f5f5; padding: 14px 20px 15px; border-top: 1px solid #ddd; -webkit-border-radius: 0 0 6px 6px; -moz-border-radius: 0 0 6px 6px; border-radius: 0 0 6px 6px; -webkit-box-shadow: inset 0 1px 0 #ffffff; -moz-box-shadow: inset 0 1px 0 #ffffff; box-shadow: inset 0 1px 0 #ffffff; zoom: 1; } .modal .modal-footer:before, .modal .modal-footer:after { display: table; content: ""; } .modal .modal-footer:after { clear: both; } .modal .modal-footer .btn { float: right; margin-left: 10px; } .twipsy { display: block; position: absolute; visibility: visible; padding: 5px; font-size: 11px; z-index: 1000; filter: alpha(opacity=80); -khtml-opacity: 0.8; -moz-opacity: 0.8; opacity: 0.8; } .twipsy.above .twipsy-arrow { bottom: 0; left: 50%; margin-left: -5px; border-left: 5px solid transparent; border-right: 5px solid transparent; border-top: 5px solid #000000; } .twipsy.left .twipsy-arrow { top: 50%; right: 0; margin-top: -5px; border-top: 5px solid transparent; border-bottom: 5px solid transparent; border-left: 5px solid #000000; } .twipsy.below .twipsy-arrow { top: 0; left: 50%; margin-left: -5px; border-left: 5px solid transparent; border-right: 5px solid transparent; border-bottom: 5px solid #000000; } .twipsy.right .twipsy-arrow { top: 50%; left: 0; margin-top: -5px; border-top: 5px solid transparent; border-bottom: 5px solid transparent; border-right: 5px solid #000000; } .twipsy .twipsy-inner { padding: 3px 8px; background-color: #000; color: white; text-align: center; max-width: 200px; text-decoration: none; -webkit-border-radius: 4px; -moz-border-radius: 4px; border-radius: 4px; } .twipsy .twipsy-arrow { position: absolute; width: 0; height: 0; } .popover { position: absolute; top: 0; left: 0; z-index: 1000; padding: 5px; display: none; } .popover.above .arrow { bottom: 0; left: 50%; margin-left: -5px; border-left: 5px solid transparent; border-right: 5px solid transparent; border-top: 5px solid #000000; } .popover.right .arrow { top: 50%; left: 0; margin-top: -5px; border-top: 5px solid transparent; border-bottom: 5px solid transparent; border-right: 5px solid #000000; } .popover.below .arrow { top: 0; left: 50%; margin-left: -5px; border-left: 5px solid transparent; border-right: 5px solid transparent; border-bottom: 5px solid #000000; } .popover.left .arrow { top: 50%; right: 0; margin-top: -5px; border-top: 5px solid transparent; border-bottom: 5px solid transparent; border-left: 5px solid #000000; } .popover .arrow { position: absolute; width: 0; height: 0; } .popover .inner { background-color: #333; background-color: rgba(0, 0, 0, 0.8); *background-color: #333; /* IE 6-7 */ padding: 3px; overflow: hidden; width: 280px; -webkit-border-radius: 6px; -moz-border-radius: 6px; border-radius: 6px; -webkit-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); -moz-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); } .popover .title { background-color: #f5f5f5; padding: 9px 15px; line-height: 1; -webkit-border-radius: 3px 3px 0 0; -moz-border-radius: 3px 3px 0 0; border-radius: 3px 3px 0 0; border-bottom: 1px solid #eee; } .popover .content { background-color: #ffffff; padding: 14px; -webkit-border-radius: 0 0 3px 3px; -moz-border-radius: 0 0 3px 3px; border-radius: 0 0 3px 3px; -webkit-background-clip: padding-box; -moz-background-clip: padding-box; background-clip: padding-box; } .popover .content p, .popover .content ul, .popover .content ol { margin-bottom: 0; } ================================================ FILE: storm-core/src/ui/public/css/style.css ================================================ /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ .js-only { display: none; } ================================================ FILE: storm-core/src/ui/public/js/script.js ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ $.tablesorter.addParser({ id:'stormtimestr', is:function (s) { return false; }, format:function (s) { if (s.search('All time') != -1) { return 1000000000; } var total = 0; $.each(s.split(' '), function (i, v) { var amt = parseInt(v); if (v.search('ms') != -1) { total += amt; } else if (v.search('s') != -1) { total += amt * 1000; } else if (v.search('m') != -1) { total += amt * 1000 * 60; } else if (v.search('h') != -1) { total += amt * 1000 * 60 * 60; } else if (v.search('d') != -1) { total += amt * 1000 * 60 * 60 * 24; } }); return total; }, type:'numeric' }); $(function () { $(".js-only").show(); }); function toggleSys() { var sys = $.cookies.get('sys') || false; sys = !sys; var exDate = new Date(); exDate.setDate(exDate.getDate() + 365); $.cookies.set('sys', sys, {'path':'/', 'expiresAt':exDate.toUTCString()}); window.location = window.location; } function ensureInt(n) { var isInt = /^\d+$/.test(n); if (!isInt) { alert("'" + n + "' is not integer."); } return isInt; } function confirmAction(id, name, action, wait, defaultWait) { var opts = { type:'POST', url:'/topology/' + id + '/' + action }; if (wait) { var waitSecs = prompt('Do you really want to ' + action + ' topology "' + name + '"? ' + 'If yes, please, specify wait time in seconds:', defaultWait); if (waitSecs != null && waitSecs != "" && ensureInt(waitSecs)) { opts.url += '/' + waitSecs; } else { return false; } } else if (!confirm('Do you really want to ' + action + ' topology "' + name + '"?')) { return false; } $("input[type=button]").attr("disabled", "disabled"); $.ajax(opts).always(function () { window.location.reload(); }).fail(function () { alert("Error while communicating with Nimbus.") }); return false; } ================================================ FILE: storm-core/test/clj/backtype/storm/clojure_test.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-test (:use [clojure test]) (:import [backtype.storm.testing TestWordSpout]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) (defbolt lalala-bolt1 ["word"] [[val :as tuple] collector] (let [ret (str val "lalala")] (emit-bolt! collector [ret] :anchor tuple) (ack! collector tuple) )) (defbolt lalala-bolt2 ["word"] {:prepare true} [conf context collector] (let [state (atom nil)] (reset! state "lalala") (bolt (execute [tuple] (let [ret (-> (.getValue tuple 0) (str @state))] (emit-bolt! collector [ret] :anchor tuple) (ack! collector tuple) )) ))) (defbolt lalala-bolt3 ["word"] {:prepare true :params [prefix]} [conf context collector] (let [state (atom nil)] (bolt (prepare [_ _ _] (reset! state (str prefix "lalala"))) (execute [{val "word" :as tuple}] (let [ret (-> (.getValue tuple 0) (str @state))] (emit-bolt! collector [ret] :anchor tuple) (ack! collector tuple) ))) )) (deftest test-clojure-bolt (with-simulated-time-local-cluster [cluster :supervisors 4] (let [nimbus (:nimbus cluster) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. false))} {"2" (thrift/mk-bolt-spec {"1" :shuffle} lalala-bolt1) "3" (thrift/mk-bolt-spec {"1" :local-or-shuffle} lalala-bolt2) "4" (thrift/mk-bolt-spec {"1" :shuffle} (lalala-bolt3 "_nathan_"))} ) results (complete-topology cluster topology :mock-sources {"1" [["david"] ["adam"] ]} )] (is (ms= [["davidlalala"] ["adamlalala"]] (read-tuples results "2"))) (is (ms= [["davidlalala"] ["adamlalala"]] (read-tuples results "3"))) (is (ms= [["david_nathan_lalala"] ["adam_nathan_lalala"]] (read-tuples results "4"))) ))) (defbolt punctuator-bolt ["word" "period" "question" "exclamation"] [tuple collector] (if (= (:word tuple) "bar") (do (emit-bolt! collector {:word "bar" :period "bar" :question "bar" "exclamation" "bar"}) (ack! collector tuple)) (let [ res (assoc tuple :period (str (:word tuple) ".")) res (assoc res :exclamation (str (:word tuple) "!")) res (assoc res :question (str (:word tuple) "?")) ] (emit-bolt! collector res) (ack! collector tuple)))) (deftest test-map-emit (with-simulated-time-local-cluster [cluster :supervisors 4] (let [topology (thrift/mk-topology {"words" (thrift/mk-spout-spec (TestWordSpout. false))} {"out" (thrift/mk-bolt-spec {"words" :shuffle} punctuator-bolt)} ) results (complete-topology cluster topology :mock-sources {"words" [["foo"] ["bar"]]} )] (is (ms= [["foo" "foo." "foo?" "foo!"] ["bar" "bar" "bar" "bar"]] (read-tuples results "out")))))) (defbolt conf-query-bolt ["conf" "val"] {:prepare true :params [conf] :conf conf} [conf context collector] (bolt (execute [tuple] (let [name (.getValue tuple 0) val (if (= name "!MAX_MSG_TIMEOUT") (.maxTopologyMessageTimeout context) (get conf name))] (emit-bolt! collector [name val] :anchor tuple) (ack! collector tuple)) ))) (deftest test-component-specific-config-clojure (with-simulated-time-local-cluster [cluster] (let [topology (topology {"1" (spout-spec (TestPlannerSpout. (Fields. ["conf"])) :conf {TOPOLOGY-MESSAGE-TIMEOUT-SECS 40}) } {"2" (bolt-spec {"1" :shuffle} (conf-query-bolt {"fake.config" 1 TOPOLOGY-MAX-TASK-PARALLELISM 2 TOPOLOGY-MAX-SPOUT-PENDING 10}) :conf {TOPOLOGY-MAX-SPOUT-PENDING 3}) }) results (complete-topology cluster topology :topology-name "test123" :storm-conf {TOPOLOGY-MAX-TASK-PARALLELISM 10 TOPOLOGY-MESSAGE-TIMEOUT-SECS 30} :mock-sources {"1" [["fake.config"] [TOPOLOGY-MAX-TASK-PARALLELISM] [TOPOLOGY-MAX-SPOUT-PENDING] ["!MAX_MSG_TIMEOUT"] [TOPOLOGY-NAME] ]})] (is (= {"fake.config" 1 TOPOLOGY-MAX-TASK-PARALLELISM 2 TOPOLOGY-MAX-SPOUT-PENDING 3 "!MAX_MSG_TIMEOUT" 40 TOPOLOGY-NAME "test123"} (->> (read-tuples results "2") (apply concat) (apply hash-map)) ))))) ================================================ FILE: storm-core/test/clj/backtype/storm/cluster_test.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-test (:import [java.util Arrays]) (:import [backtype.storm.daemon.common Assignment StormBase SupervisorInfo]) (:use [clojure test]) (:use [backtype.storm cluster config util testing])) (defn mk-config [zk-port] (merge (read-storm-config) {STORM-ZOOKEEPER-PORT zk-port STORM-ZOOKEEPER-SERVERS ["localhost"]})) (defn mk-state ([zk-port] (mk-distributed-cluster-state (mk-config zk-port))) ([zk-port cb] (let [ret (mk-state zk-port)] (.register ret cb) ret ))) (defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port))) (deftest test-basics (with-inprocess-zookeeper zk-port (let [state (mk-state zk-port)] (.set-data state "/root" (barr 1 2 3)) (is (Arrays/equals (barr 1 2 3) (.get-data state "/root" false))) (is (= nil (.get-data state "/a" false))) (.set-data state "/root/a" (barr 1 2)) (.set-data state "/root" (barr 1)) (is (Arrays/equals (barr 1) (.get-data state "/root" false))) (is (Arrays/equals (barr 1 2) (.get-data state "/root/a" false))) (.set-data state "/a/b/c/d" (barr 99)) (is (Arrays/equals (barr 99) (.get-data state "/a/b/c/d" false))) (.mkdirs state "/lalala") (is (= [] (.get-children state "/lalala" false))) (is (= #{"root" "a" "lalala"} (set (.get-children state "/" false)))) (.delete-node state "/a") (is (= #{"root" "lalala"} (set (.get-children state "/" false)))) (is (= nil (.get-data state "/a/b/c/d" false))) (.close state) ))) (deftest test-multi-state (with-inprocess-zookeeper zk-port (let [state1 (mk-state zk-port) state2 (mk-state zk-port)] (.set-data state1 "/root" (barr 1)) (is (Arrays/equals (barr 1) (.get-data state1 "/root" false))) (is (Arrays/equals (barr 1) (.get-data state2 "/root" false))) (.delete-node state2 "/root") (is (= nil (.get-data state1 "/root" false))) (is (= nil (.get-data state2 "/root" false))) (.close state1) (.close state2) ))) (deftest test-ephemeral (with-inprocess-zookeeper zk-port (let [state1 (mk-state zk-port) state2 (mk-state zk-port) state3 (mk-state zk-port)] (.set-ephemeral-node state1 "/a" (barr 1)) (is (Arrays/equals (barr 1) (.get-data state1 "/a" false))) (is (Arrays/equals (barr 1) (.get-data state2 "/a" false))) (.close state3) (is (Arrays/equals (barr 1) (.get-data state1 "/a" false))) (is (Arrays/equals (barr 1) (.get-data state2 "/a" false))) (.close state1) (is (= nil (.get-data state2 "/a" false))) (.close state2) ))) (defn mk-callback-tester [] (let [last (atom nil) cb (fn [type path] (reset! last {:type type :path path}))] [last cb] )) (defn read-and-reset! [aatom] (let [time (System/currentTimeMillis)] (loop [] (if-let [val @aatom] (do (reset! aatom nil) val) (do (when (> (- (System/currentTimeMillis) time) 30000) (throw (RuntimeException. "Waited too long for atom to change state"))) (Thread/sleep 10) (recur)) )))) (deftest test-callbacks (with-inprocess-zookeeper zk-port (let [[state1-last-cb state1-cb] (mk-callback-tester) state1 (mk-state zk-port state1-cb) [state2-last-cb state2-cb] (mk-callback-tester) state2 (mk-state zk-port state2-cb)] (.set-data state1 "/root" (barr 1)) (.get-data state2 "/root" true) (is (= nil @state1-last-cb)) (is (= nil @state2-last-cb)) (.set-data state2 "/root" (barr 2)) (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb))) (is (= nil @state1-last-cb)) (.set-data state2 "/root" (barr 3)) (is (= nil @state2-last-cb)) (.get-data state2 "/root" true) (.get-data state2 "/root" false) (.delete-node state1 "/root") (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb))) (.get-data state2 "/root" true) (.set-ephemeral-node state1 "/root" (barr 1 2 3 4)) (is (= {:type :node-created :path "/root"} (read-and-reset! state2-last-cb))) (.get-children state1 "/" true) (.set-data state2 "/a" (barr 9)) (is (= nil @state2-last-cb)) (is (= {:type :node-children-changed :path "/"} (read-and-reset! state1-last-cb))) (.get-data state2 "/root" true) (.set-ephemeral-node state1 "/root" (barr 1 2)) (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb))) (.mkdirs state1 "/ccc") (.get-children state1 "/ccc" true) (.get-data state2 "/ccc/b" true) (.set-data state2 "/ccc/b" (barr 8)) (is (= {:type :node-created :path "/ccc/b"} (read-and-reset! state2-last-cb))) (is (= {:type :node-children-changed :path "/ccc"} (read-and-reset! state1-last-cb))) (.get-data state2 "/root" true) (.get-data state2 "/root2" true) (.close state1) (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb))) (.set-data state2 "/root2" (barr 9)) (is (= {:type :node-created :path "/root2"} (read-and-reset! state2-last-cb))) (.close state2) ))) (deftest test-storm-cluster-state-basics (with-inprocess-zookeeper zk-port (let [state (mk-storm-state zk-port) assignment1 (Assignment. "/aaa" {} {1 [2 2002 1]} {}) assignment2 (Assignment. "/aaa" {} {1 [2 2002]} {}) base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {}) base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {})] (is (= [] (.assignments state nil))) (.set-assignment! state "storm1" assignment1) (is (= assignment1 (.assignment-info state "storm1" nil))) (is (= nil (.assignment-info state "storm3" nil))) (.set-assignment! state "storm1" assignment2) (.set-assignment! state "storm3" assignment1) (is (= #{"storm1" "storm3"} (set (.assignments state nil)))) (is (= assignment2 (.assignment-info state "storm1" nil))) (is (= assignment1 (.assignment-info state "storm3" nil))) (is (= [] (.active-storms state))) (.activate-storm! state "storm1" base1) (is (= ["storm1"] (.active-storms state))) (is (= base1 (.storm-base state "storm1" nil))) (is (= nil (.storm-base state "storm2" nil))) (.activate-storm! state "storm2" base2) (is (= base1 (.storm-base state "storm1" nil))) (is (= base2 (.storm-base state "storm2" nil))) (is (= #{"storm1" "storm2"} (set (.active-storms state)))) (.remove-storm-base! state "storm1") (is (= base2 (.storm-base state "storm2" nil))) (is (= #{"storm2"} (set (.active-storms state)))) ;; TODO add tests for task info and task heartbeat setting and getting (.disconnect state) ))) (defn- validate-errors! [state storm-id component errors-list] (let [errors (.errors state storm-id component)] ;;(println errors) (is (= (count errors) (count errors-list))) (doseq [[error target] (map vector errors errors-list)] (when-not (.contains (:error error) target) (println target " => " (:error error))) (is (.contains (:error error) target)) ))) (deftest test-storm-cluster-state-errors (with-inprocess-zookeeper zk-port (with-simulated-time (let [state (mk-storm-state zk-port)] (.report-error state "a" "1" (RuntimeException.)) (validate-errors! state "a" "1" ["RuntimeException"]) (.report-error state "a" "1" (IllegalArgumentException.)) (validate-errors! state "a" "1" ["RuntimeException" "IllegalArgumentException"]) (doseq [i (range 10)] (.report-error state "a" "2" (RuntimeException.)) (advance-time-secs! 2)) (validate-errors! state "a" "2" (repeat 10 "RuntimeException")) (doseq [i (range 5)] (.report-error state "a" "2" (IllegalArgumentException.)) (advance-time-secs! 2)) (validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException") (repeat 5 "RuntimeException") )) (.disconnect state) )))) (deftest test-supervisor-state (with-inprocess-zookeeper zk-port (let [state1 (mk-storm-state zk-port) state2 (mk-storm-state zk-port)] (is (= [] (.supervisors state1 nil))) (.supervisor-heartbeat! state2 "2" {:a 1}) (.supervisor-heartbeat! state1 "1" {}) (is (= {:a 1} (.supervisor-info state1 "2"))) (is (= {} (.supervisor-info state1 "1"))) (is (= #{"1" "2"} (set (.supervisors state1 nil)))) (is (= #{"1" "2"} (set (.supervisors state2 nil)))) (.disconnect state2) (is (= #{"1"} (set (.supervisors state1 nil)))) (.disconnect state1) ))) (deftest test-storm-state-callbacks ;; TODO finish ) ================================================ FILE: storm-core/test/clj/backtype/storm/config_test.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-test (:import [backtype.storm Config ConfigValidation]) (:import [backtype.storm.scheduler TopologyDetails]) (:import [backtype.storm.utils Utils]) (:use [clojure test]) (:use [backtype.storm config util]) ) (deftest test-validity (is (Utils/isValidConf {TOPOLOGY-DEBUG true "q" "asasdasd" "aaa" (Integer. "123") "bbb" (Long. "456") "eee" [1 2 (Integer. "3") (Long. "4")]})) (is (not (Utils/isValidConf {"qqq" (backtype.storm.utils.Utils.)}))) ) (deftest test-power-of-2-validator (let [validator ConfigValidation/PowerOf2Validator] (doseq [x [42.42 42 23423423423 -33 -32 -1 -0.00001 0 -0 "Forty-two"]] (is (thrown-cause? java.lang.IllegalArgumentException (.validateField validator "test" x)))) (doseq [x [64 4294967296 1 nil]] (is (nil? (try (.validateField validator "test" x) (catch Exception e e))))))) (deftest test-list-validator (let [validator ConfigValidation/StringsValidator] (doseq [x [ ["Forty-two" 42] [42] [true "false"] [nil] [nil "nil"] ]] (is (thrown-cause-with-msg? java.lang.IllegalArgumentException #"(?i).*each element.*" (.validateField validator "test" x)))) (doseq [x ["not a list at all"]] (is (thrown-cause-with-msg? java.lang.IllegalArgumentException #"(?i).*must be an iterable.*" (.validateField validator "test" x)))) (doseq [x [ ["one" "two" "three"] [""] ["42" "64"] nil ]] (is (nil? (try (.validateField validator "test" x) (catch Exception e e))))))) (deftest test-topology-workers-is-number (let [validator (CONFIG-SCHEMA-MAP TOPOLOGY-WORKERS)] (.validateField validator "test" 42) ;; The float can be rounded down to an int. (.validateField validator "test" 3.14159) (is (thrown-cause? java.lang.IllegalArgumentException (.validateField validator "test" "42"))))) (deftest test-isolation-scheduler-machines-is-map (let [validator (CONFIG-SCHEMA-MAP ISOLATION-SCHEDULER-MACHINES)] (is (nil? (try (.validateField validator "test" {}) (catch Exception e e)))) (is (nil? (try (.validateField validator "test" {"host0" 1 "host1" 2}) (catch Exception e e)))) (is (thrown-cause? java.lang.IllegalArgumentException (.validateField validator "test" 42))))) ================================================ FILE: storm-core/test/clj/backtype/storm/drpc_test.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.drpc-test (:use [clojure test]) (:import [backtype.storm.drpc ReturnResults DRPCSpout LinearDRPCTopologyBuilder]) (:import [backtype.storm.topology FailedException]) (:import [backtype.storm.coordination CoordinatedBolt$FinishedCallback]) (:import [backtype.storm LocalDRPC LocalCluster]) (:import [backtype.storm.tuple Fields]) (:import [backtype.storm.generated DRPCExecutionException]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) (:use [backtype.storm clojure]) ) (bootstrap) (defbolt exclamation-bolt ["result" "return-info"] [tuple collector] (emit-bolt! collector [(str (.getString tuple 0) "!!!") (.getValue tuple 1)] :anchor tuple) (ack! collector tuple) ) (deftest test-drpc-flow (let [drpc (LocalDRPC.) spout (DRPCSpout. "test" drpc) cluster (LocalCluster.) topology (topology {"1" (spout-spec spout)} {"2" (bolt-spec {"1" :shuffle} exclamation-bolt) "3" (bolt-spec {"2" :shuffle} (ReturnResults.))})] (.submitTopology cluster "test" {} topology) (is (= "aaa!!!" (.execute drpc "test" "aaa"))) (is (= "b!!!" (.execute drpc "test" "b"))) (is (= "c!!!" (.execute drpc "test" "c"))) (.shutdown cluster) (.shutdown drpc) )) (defbolt exclamation-bolt-drpc ["id" "result"] [tuple collector] (emit-bolt! collector [(.getValue tuple 0) (str (.getString tuple 1) "!!!")] :anchor tuple) (ack! collector tuple) ) (deftest test-drpc-builder (let [drpc (LocalDRPC.) cluster (LocalCluster.) builder (LinearDRPCTopologyBuilder. "test") ] (.addBolt builder exclamation-bolt-drpc 3) (.submitTopology cluster "builder-test" {} (.createLocalTopology builder drpc)) (is (= "aaa!!!" (.execute drpc "test" "aaa"))) (is (= "b!!!" (.execute drpc "test" "b"))) (is (= "c!!!" (.execute drpc "test" "c"))) (.shutdown cluster) (.shutdown drpc) )) (defn safe-inc [v] (if v (inc v) 1)) (defbolt partial-count ["request" "count"] {:prepare true} [conf context collector] (let [counts (atom {})] (bolt (execute [tuple] (let [id (.getValue tuple 0)] (swap! counts update-in [id] safe-inc) (ack! collector tuple) )) CoordinatedBolt$FinishedCallback (finishedId [this id] (emit-bolt! collector [id (get @counts id 0)]) )) )) (defn safe+ [v1 v2] (if v1 (+ v1 v2) v2)) (defbolt count-aggregator ["request" "total"] {:prepare true} [conf context collector] (let [counts (atom {})] (bolt (execute [tuple] (let [id (.getValue tuple 0) count (.getValue tuple 1)] (swap! counts update-in [id] safe+ count) (ack! collector tuple) )) CoordinatedBolt$FinishedCallback (finishedId [this id] (emit-bolt! collector [id (get @counts id 0)]) )) )) (defbolt create-tuples ["request"] [tuple collector] (let [id (.getValue tuple 0) amt (Integer/parseInt (.getValue tuple 1))] (doseq [i (range (* amt amt))] (emit-bolt! collector [id] :anchor tuple)) (ack! collector tuple) )) (deftest test-drpc-coordination (let [drpc (LocalDRPC.) cluster (LocalCluster.) builder (LinearDRPCTopologyBuilder. "square") ] (.addBolt builder create-tuples 3) (doto (.addBolt builder partial-count 3) (.shuffleGrouping)) (doto (.addBolt builder count-aggregator 3) (.fieldsGrouping (Fields. ["request"]))) (.submitTopology cluster "squared" {} (.createLocalTopology builder drpc)) (is (= "4" (.execute drpc "square" "2"))) (is (= "100" (.execute drpc "square" "10"))) (is (= "1" (.execute drpc "square" "1"))) (is (= "0" (.execute drpc "square" "0"))) (.shutdown cluster) (.shutdown drpc) )) (defbolt id-bolt ["request" "val"] [tuple collector] (emit-bolt! collector (.getValues tuple) :anchor tuple) (ack! collector tuple)) (defbolt emit-finish ["request" "result"] {:prepare true} [conf context collector] (bolt (execute [tuple] (ack! collector tuple) ) CoordinatedBolt$FinishedCallback (finishedId [this id] (emit-bolt! collector [id "done"]) ))) (deftest test-drpc-coordination-tricky (let [drpc (LocalDRPC.) cluster (LocalCluster.) builder (LinearDRPCTopologyBuilder. "tricky") ] (.addBolt builder id-bolt 3) (doto (.addBolt builder id-bolt 3) (.shuffleGrouping)) (doto (.addBolt builder emit-finish 3) (.fieldsGrouping (Fields. ["request"]))) (.submitTopology cluster "tricky" {} (.createLocalTopology builder drpc)) (is (= "done" (.execute drpc "tricky" "2"))) (is (= "done" (.execute drpc "tricky" "3"))) (is (= "done" (.execute drpc "tricky" "4"))) (.shutdown cluster) (.shutdown drpc) )) (defbolt fail-finish-bolt ["request" "result"] {:prepare true} [conf context collector] (bolt (execute [tuple] (ack! collector tuple)) CoordinatedBolt$FinishedCallback (finishedId [this id] (throw (FailedException.)) ))) (deftest test-drpc-fail-finish (let [drpc (LocalDRPC.) cluster (LocalCluster.) builder (LinearDRPCTopologyBuilder. "fail2") ] (.addBolt builder fail-finish-bolt 3) (.submitTopology cluster "fail2" {} (.createLocalTopology builder drpc)) (is (thrown? DRPCExecutionException (.execute drpc "fail2" "2"))) (.shutdown cluster) (.shutdown drpc) )) ================================================ FILE: storm-core/test/clj/backtype/storm/fields_test.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.fields-test (:use [clojure test]) (:import [backtype.storm.tuple Fields]) (:import [java.util List]) (:import [java.util Iterator])) (deftest test-fields-constructor (testing "constructor" (testing "with (String... fields)" (is (instance? Fields (Fields. (into-array String '("foo" "bar"))))) (is (thrown? IllegalArgumentException (Fields. (into-array String '("foo" "bar" "foo")))))) (testing "with (List fields)" (is (instance? Fields (Fields. '("foo" "bar")))) (is (thrown? IllegalArgumentException (Fields. '("foo" "bar" "foo"))))))) (deftest test-fields-methods (let [fields (Fields. '("foo" "bar"))] (testing "method" (testing ".size" (is (= (.size fields) 2))) (testing ".get" (is (= (.get fields 0) "foo")) (is (= (.get fields 1) "bar")) (is (thrown? IndexOutOfBoundsException (.get fields 2)))) (testing ".fieldIndex" (is (= (.fieldIndex fields "foo") 0)) (is (= (.fieldIndex fields "bar") 1)) (is (thrown? IllegalArgumentException (.fieldIndex fields "baz")))) (testing ".contains" (is (= (.contains fields "foo") true)) (is (= (.contains fields "bar") true)) (is (= (.contains fields "baz") false))) (testing ".toList" (is (instance? List (.toList fields))) (is (= (count (.toList fields)) 2)) (is (not-any? false? (map = (.toList fields) '("foo" "bar"))))) (testing ".iterator" (is (instance? Iterator (.iterator fields))) (is (= (count (iterator-seq (.iterator fields))) 2)) (is (not-any? false? (map = (iterator-seq (.iterator fields)) '("foo" "bar"))))) (testing ".select" (is (instance? List (.select fields (Fields. '("bar")) '("a" "b" "c")))) (is (= (.select fields (Fields. '("bar")) '("a" "b" "c")) '("b"))))))) ================================================ FILE: storm-core/test/clj/backtype/storm/grouping_test.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.grouping-test (:use [clojure test]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter NGrouping]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) (deftest test-shuffle (with-simulated-time-local-cluster [cluster :supervisors 4] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)} {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.) :parallelism-hint 6) }) results (complete-topology cluster topology ;; important for test that ;; #tuples = multiple of 4 and 6 :mock-sources {"1" [["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ]} )] (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]])) (read-tuples results "2"))) ))) (defbolt id-bolt ["val"] [tuple collector] (emit-bolt! collector (.getValues tuple)) (ack! collector tuple)) (deftest test-custom-groupings (with-simulated-time-local-cluster [cluster] (let [topology (topology {"1" (spout-spec (TestWordSpout. true))} {"2" (bolt-spec {"1" (NGrouping. 2)} id-bolt :p 4) "3" (bolt-spec {"1" (JavaObject. "backtype.storm.testing.NGrouping" [(JavaObjectArg/int_arg 3)])} id-bolt :p 6) }) results (complete-topology cluster topology :mock-sources {"1" [["a"] ["b"] ]} )] (is (ms= [["a"] ["a"] ["b"] ["b"]] (read-tuples results "2"))) (is (ms= [["a"] ["a"] ["a"] ["b"] ["b"] ["b"]] (read-tuples results "3"))) ))) ================================================ FILE: storm-core/test/clj/backtype/storm/integration_test.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.integration-test (:use [clojure test]) (:import [backtype.storm.topology TopologyBuilder]) (:import [backtype.storm.generated InvalidTopologyException SubmitOptions TopologyInitialStatus]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestConfBolt AckFailMapTracker]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) (deftest test-basic-topology (doseq [zmq-on? [true false]] (with-simulated-time-local-cluster [cluster :supervisors 4 :daemon-conf {STORM-LOCAL-MODE-ZMQ zmq-on?}] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :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.)) }) results (complete-topology cluster topology :mock-sources {"1" [["nathan"] ["bob"] ["joey"] ["nathan"]]} :storm-conf {TOPOLOGY-WORKERS 2})] (is (ms= [["nathan"] ["bob"] ["joey"] ["nathan"]] (read-tuples results "1"))) (is (ms= [["nathan" 1] ["nathan" 2] ["bob" 1] ["joey" 1]] (read-tuples results "2"))) (is (= [[1] [2] [3] [4]] (read-tuples results "3"))) (is (= [[1] [2] [3] [4]] (read-tuples results "4"))) )))) (defbolt emit-task-id ["tid"] {:prepare true} [conf context collector] (let [tid (.getThisTaskIndex context)] (bolt (execute [tuple] (emit-bolt! collector [tid] :anchor tuple) (ack! collector tuple) )))) (deftest test-multi-tasks-per-executor (with-simulated-time-local-cluster [cluster :supervisors 4] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true))} {"2" (thrift/mk-bolt-spec {"1" :shuffle} emit-task-id :parallelism-hint 3 :conf {TOPOLOGY-TASKS 6}) }) results (complete-topology cluster topology :mock-sources {"1" [["a"] ["a"] ["a"] ["a"] ["a"] ["a"]]})] (is (ms= [[0] [1] [2] [3] [4] [5]] (read-tuples results "2"))) ))) (defbolt ack-every-other {} {:prepare true} [conf context collector] (let [state (atom -1)] (bolt (execute [tuple] (let [val (swap! state -)] (when (pos? val) (ack! collector tuple) )))))) (defn assert-loop [afn ids] (while (not (every? afn ids)) (Thread/sleep 1))) (defn assert-acked [tracker & ids] (assert-loop #(.isAcked tracker %) ids)) (defn assert-failed [tracker & ids] (assert-loop #(.isFailed tracker %) ids)) (deftest test-timeout (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true}] (let [feeder (feeder-spout ["field1"]) tracker (AckFailMapTracker.) _ (.setAckFailDelegate feeder tracker) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec feeder)} {"2" (thrift/mk-bolt-spec {"1" :global} ack-every-other)})] (submit-local-topology (:nimbus cluster) "timeout-tester" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10} topology) (.feed feeder ["a"] 1) (.feed feeder ["b"] 2) (.feed feeder ["c"] 3) (advance-cluster-time cluster 9) (assert-acked tracker 1 3) (is (not (.isFailed tracker 2))) (advance-cluster-time cluster 12) (assert-failed tracker 2) ))) (defn mk-validate-topology-1 [] (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 3)} {"2" (thrift/mk-bolt-spec {"1" ["word"]} (TestWordCounter.) :parallelism-hint 4)})) (defn mk-invalidate-topology-1 [] (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 3)} {"2" (thrift/mk-bolt-spec {"3" ["word"]} (TestWordCounter.) :parallelism-hint 4)})) (defn mk-invalidate-topology-2 [] (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 3)} {"2" (thrift/mk-bolt-spec {"1" ["non-exists-field"]} (TestWordCounter.) :parallelism-hint 4)})) (defn mk-invalidate-topology-3 [] (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 3)} {"2" (thrift/mk-bolt-spec {["1" "non-exists-stream"] ["word"]} (TestWordCounter.) :parallelism-hint 4)})) (defn try-complete-wc-topology [cluster topology] (try (do (complete-topology cluster topology :mock-sources {"1" [["nathan"] ["bob"] ["joey"] ["nathan"]]} :storm-conf {TOPOLOGY-WORKERS 2}) false) (catch InvalidTopologyException e true))) (deftest test-validate-topology-structure (with-simulated-time-local-cluster [cluster :supervisors 4] (let [any-error1? (try-complete-wc-topology cluster (mk-validate-topology-1)) any-error2? (try-complete-wc-topology cluster (mk-invalidate-topology-1)) any-error3? (try-complete-wc-topology cluster (mk-invalidate-topology-2)) any-error4? (try-complete-wc-topology cluster (mk-invalidate-topology-3))] (is (= any-error1? false)) (is (= any-error2? true)) (is (= any-error3? true)) (is (= any-error4? true))))) (defbolt identity-bolt ["num"] [tuple collector] (emit-bolt! collector (.getValues tuple) :anchor tuple) (ack! collector tuple)) (deftest test-system-stream ;; this test works because mocking a spout splits up the tuples evenly among the tasks (with-simulated-time-local-cluster [cluster] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :p 3)} {"2" (thrift/mk-bolt-spec {"1" ["word"] ["1" "__system"] :global} identity-bolt :p 1) }) results (complete-topology cluster topology :mock-sources {"1" [["a"] ["b"] ["c"]]} :storm-conf {TOPOLOGY-WORKERS 2})] (is (ms= [["a"] ["b"] ["c"] ["startup"] ["startup"] ["startup"]] (read-tuples results "2"))) ))) (defn ack-tracking-feeder [fields] (let [tracker (AckTracker.)] [(doto (feeder-spout fields) (.setAckFailDelegate tracker)) (fn [val] (is (= (.getNumAcks tracker) val)) (.resetNumAcks tracker) )] )) (defbolt branching-bolt ["num"] {:params [amt]} [tuple collector] (doseq [i (range amt)] (emit-bolt! collector [i] :anchor tuple)) (ack! collector tuple)) (defbolt agg-bolt ["num"] {:prepare true :params [amt]} [conf context collector] (let [seen (atom [])] (bolt (execute [tuple] (swap! seen conj tuple) (when (= (count @seen) amt) (emit-bolt! collector [1] :anchor @seen) (doseq [s @seen] (ack! collector s)) (reset! seen []) ))) )) (defbolt ack-bolt {} [tuple collector] (ack! collector tuple)) (deftest test-acking (with-tracked-cluster [cluster] (let [[feeder1 checker1] (ack-tracking-feeder ["num"]) [feeder2 checker2] (ack-tracking-feeder ["num"]) [feeder3 checker3] (ack-tracking-feeder ["num"]) tracked (mk-tracked-topology cluster (topology {"1" (spout-spec feeder1) "2" (spout-spec feeder2) "3" (spout-spec feeder3)} {"4" (bolt-spec {"1" :shuffle} (branching-bolt 2)) "5" (bolt-spec {"2" :shuffle} (branching-bolt 4)) "6" (bolt-spec {"3" :shuffle} (branching-bolt 1)) "7" (bolt-spec {"4" :shuffle "5" :shuffle "6" :shuffle} (agg-bolt 3)) "8" (bolt-spec {"7" :shuffle} (branching-bolt 2)) "9" (bolt-spec {"8" :shuffle} ack-bolt)} ))] (submit-local-topology (:nimbus cluster) "acking-test1" {} (:topology tracked)) (.feed feeder1 [1]) (tracked-wait tracked 1) (checker1 0) (.feed feeder2 [1]) (tracked-wait tracked 1) (checker1 1) (checker2 1) (.feed feeder1 [1]) (tracked-wait tracked 1) (checker1 0) (.feed feeder1 [1]) (tracked-wait tracked 1) (checker1 1) (.feed feeder3 [1]) (tracked-wait tracked 1) (checker1 0) (checker3 0) (.feed feeder2 [1]) (tracked-wait tracked 1) (checker1 1) (checker2 1) (checker3 1) ))) (deftest test-ack-branching (with-tracked-cluster [cluster] (let [[feeder checker] (ack-tracking-feeder ["num"]) tracked (mk-tracked-topology cluster (topology {"1" (spout-spec feeder)} {"2" (bolt-spec {"1" :shuffle} identity-bolt) "3" (bolt-spec {"1" :shuffle} identity-bolt) "4" (bolt-spec {"2" :shuffle "3" :shuffle} (agg-bolt 4))}))] (submit-local-topology (:nimbus cluster) "test-acking2" {} (:topology tracked)) (.feed feeder [1]) (tracked-wait tracked 1) (checker 0) (.feed feeder [1]) (tracked-wait tracked 1) (checker 2) ))) (defbolt dup-anchor ["num"] [tuple collector] (emit-bolt! collector [1] :anchor [tuple tuple]) (ack! collector tuple)) (def bolt-prepared? (atom false)) (defbolt prepare-tracked-bolt [] {:prepare true} [conf context collector] (reset! bolt-prepared? true) (bolt (execute [tuple] (ack! collector tuple)))) (def spout-opened? (atom false)) (defspout open-tracked-spout ["val"] [conf context collector] (reset! spout-opened? true) (spout (nextTuple []))) (deftest test-submit-inactive-topology (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true}] (let [feeder (feeder-spout ["field1"]) tracker (AckFailMapTracker.) _ (.setAckFailDelegate feeder tracker) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec feeder) "2" (thrift/mk-spout-spec open-tracked-spout)} {"3" (thrift/mk-bolt-spec {"1" :global} prepare-tracked-bolt)})] (reset! bolt-prepared? false) (reset! spout-opened? false) (submit-local-topology-with-opts (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10} topology (SubmitOptions. TopologyInitialStatus/INACTIVE)) (.feed feeder ["a"] 1) (advance-cluster-time cluster 9) (is (not @bolt-prepared?)) (is (not @spout-opened?)) (.activate (:nimbus cluster) "test") (advance-cluster-time cluster 12) (assert-acked tracker 1) (is @bolt-prepared?) (is @spout-opened?)))) (deftest test-acking-self-anchor (with-tracked-cluster [cluster] (let [[feeder checker] (ack-tracking-feeder ["num"]) tracked (mk-tracked-topology cluster (topology {"1" (spout-spec feeder)} {"2" (bolt-spec {"1" :shuffle} dup-anchor) "3" (bolt-spec {"2" :shuffle} ack-bolt)}))] (submit-local-topology (:nimbus cluster) "test" {} (:topology tracked)) (.feed feeder [1]) (tracked-wait tracked 1) (checker 1) (.feed feeder [1]) (.feed feeder [1]) (.feed feeder [1]) (tracked-wait tracked 3) (checker 3) ))) ;; (defspout ConstantSpout ["val"] {:prepare false} ;; [collector] ;; (Time/sleep 100) ;; (emit-spout! collector [1])) ;; (def errored (atom false)) ;; (def restarted (atom false)) ;; (defbolt local-error-checker {} [tuple collector] ;; (when-not @errored ;; (reset! errored true) ;; (println "erroring") ;; (throw (RuntimeException.))) ;; (when-not @restarted (println "restarted")) ;; (reset! restarted true)) ;; (deftest test-no-halt-local-mode ;; (with-simulated-time-local-cluster [cluster] ;; (let [topology (topology ;; {1 (spout-spec ConstantSpout)} ;; {2 (bolt-spec {1 :shuffle} local-error-checker) ;; })] ;; (submit-local-topology (:nimbus cluster) ;; "test" ;; {} ;; topology) ;; (while (not @restarted) ;; (advance-time-ms! 100)) ;; ))) (defspout IncSpout ["word"] [conf context collector] (let [state (atom 0)] (spout (nextTuple [] (Thread/sleep 100) (emit-spout! collector [@state] :id 1) ) (ack [id] (swap! state inc)) ))) (defspout IncSpout2 ["word"] {:params [prefix]} [conf context collector] (let [state (atom 0)] (spout (nextTuple [] (Thread/sleep 100) (swap! state inc) (emit-spout! collector [(str prefix "-" @state)]) ) ))) ;; (deftest test-clojure-spout ;; (with-local-cluster [cluster] ;; (let [nimbus (:nimbus cluster) ;; top (topology ;; {1 (spout-spec IncSpout)} ;; {} ;; )] ;; (submit-local-topology nimbus ;; "spout-test" ;; {TOPOLOGY-DEBUG true ;; TOPOLOGY-MESSAGE-TIMEOUT-SECS 3} ;; top) ;; (Thread/sleep 10000) ;; (.killTopology nimbus "spout-test") ;; (Thread/sleep 10000) ;; ))) (deftest test-kryo-decorators-config (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-OPTIMIZE false TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS true TOPOLOGY-KRYO-DECORATORS ["this-is-overriden"]}] (letlocals (bind builder (TopologyBuilder.)) (.setSpout builder "1" (TestPlannerSpout. (Fields. ["conf"]))) (-> builder (.setBolt "2" (TestConfBolt. {TOPOLOGY-KRYO-DECORATORS ["one" "two"]})) (.shuffleGrouping "1")) (bind results (complete-topology cluster (.createTopology builder) :storm-conf {TOPOLOGY-KRYO-DECORATORS ["one" "three"]} :mock-sources {"1" [[TOPOLOGY-KRYO-DECORATORS]]})) (is (= {"topology.kryo.decorators" (list "one" "two" "three")} (->> (read-tuples results "2") (apply concat) (apply hash-map))))))) (deftest test-component-specific-config (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-OPTIMIZE false TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS true}] (letlocals (bind builder (TopologyBuilder.)) (.setSpout builder "1" (TestPlannerSpout. (Fields. ["conf"]))) (-> builder (.setBolt "2" (TestConfBolt. {"fake.config" 123 TOPOLOGY-MAX-TASK-PARALLELISM 20 TOPOLOGY-MAX-SPOUT-PENDING 30 TOPOLOGY-OPTIMIZE true TOPOLOGY-KRYO-REGISTER [{"fake.type" "bad.serializer"} {"fake.type2" "a.serializer"}] })) (.shuffleGrouping "1") (.setMaxTaskParallelism (int 2)) (.addConfiguration "fake.config2" 987) ) (bind results (complete-topology cluster (.createTopology builder) :storm-conf {TOPOLOGY-KRYO-REGISTER [{"fake.type" "good.serializer" "fake.type3" "a.serializer3"}]} :mock-sources {"1" [["fake.config"] [TOPOLOGY-MAX-TASK-PARALLELISM] [TOPOLOGY-MAX-SPOUT-PENDING] [TOPOLOGY-OPTIMIZE] ["fake.config2"] [TOPOLOGY-KRYO-REGISTER] ]})) (is (= {"fake.config" 123 "fake.config2" 987 TOPOLOGY-MAX-TASK-PARALLELISM 2 TOPOLOGY-MAX-SPOUT-PENDING 30 TOPOLOGY-OPTIMIZE false TOPOLOGY-KRYO-REGISTER {"fake.type" "good.serializer" "fake.type2" "a.serializer" "fake.type3" "a.serializer3"}} (->> (read-tuples results "2") (apply concat) (apply hash-map)) )) ))) (defbolt hooks-bolt ["emit" "ack" "fail" "executed"] {:prepare true} [conf context collector] (let [acked (atom 0) failed (atom 0) executed (atom 0) emitted (atom 0)] (.addTaskHook context (reify backtype.storm.hooks.ITaskHook (prepare [this conf context] ) (cleanup [this] ) (emit [this info] (swap! emitted inc)) (boltAck [this info] (swap! acked inc)) (boltFail [this info] (swap! failed inc)) (boltExecute [this info] (swap! executed inc)) )) (bolt (execute [tuple] (emit-bolt! collector [@emitted @acked @failed @executed]) (if (= 0 (- @acked @failed)) (ack! collector tuple) (fail! collector tuple)) )))) (deftest test-hooks (with-simulated-time-local-cluster [cluster] (let [topology (topology {"1" (spout-spec (TestPlannerSpout. (Fields. ["conf"]))) } {"2" (bolt-spec {"1" :shuffle} hooks-bolt) }) results (complete-topology cluster topology :mock-sources {"1" [[1] [1] [1] [1] ]})] (is (= [[0 0 0 0] [2 1 0 1] [4 1 1 2] [6 2 1 3]] (read-tuples results "2") ))))) (defbolt report-errors-bolt {} [tuple collector] (doseq [i (range (.getValue tuple 0))] (report-error! collector (RuntimeException.))) (ack! collector tuple)) (deftest test-throttled-errors (with-simulated-time (with-tracked-cluster [cluster] (let [state (:storm-cluster-state cluster) [feeder checker] (ack-tracking-feeder ["num"]) tracked (mk-tracked-topology cluster (topology {"1" (spout-spec feeder)} {"2" (bolt-spec {"1" :shuffle} report-errors-bolt)})) _ (submit-local-topology (:nimbus cluster) "test-errors" {TOPOLOGY-ERROR-THROTTLE-INTERVAL-SECS 10 TOPOLOGY-MAX-ERROR-REPORT-PER-INTERVAL 4 TOPOLOGY-DEBUG true } (:topology tracked)) storm-id (get-storm-id state "test-errors") errors-count (fn [] (count (.errors state storm-id "2")))] ;; so it launches the topology (advance-cluster-time cluster 2) (.feed feeder [6]) (tracked-wait tracked 1) (is (= 4 (errors-count))) (advance-time-secs! 5) (.feed feeder [2]) (tracked-wait tracked 1) (is (= 4 (errors-count))) (advance-time-secs! 6) (.feed feeder [2]) (tracked-wait tracked 1) (is (= 6 (errors-count))) (advance-time-secs! 6) (.feed feeder [3]) (tracked-wait tracked 1) (is (= 8 (errors-count))) )))) (deftest test-acking-branching-complex ;; test acking with branching in the topology ) (deftest test-fields-grouping ;; 1. put a shitload of random tuples through it and test that counts are right ;; 2. test that different spouts with different phints group the same way ) (deftest test-all-grouping ) (deftest test-direct-grouping ) ================================================ FILE: storm-core/test/clj/backtype/storm/local_state_test.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.local-state-test (:use [clojure test]) (:use [backtype.storm testing]) (:import [backtype.storm.utils LocalState])) (deftest test-local-state (with-local-tmp [dir1 dir2] (let [ls1 (LocalState. dir1) ls2 (LocalState. dir2)] (is (= {} (.snapshot ls1))) (.put ls1 "a" 1) (.put ls1 "b" 2) (is (= {"a" 1 "b" 2} (.snapshot ls1))) (is (= {} (.snapshot ls2))) (is (= 1 (.get ls1 "a"))) (is (= nil (.get ls1 "c"))) (is (= 2 (.get ls1 "b"))) (is (= {"a" 1 "b" 2} (.snapshot (LocalState. dir1)))) (.put ls2 "b" 1) (.put ls2 "b" 2) (.put ls2 "b" 3) (.put ls2 "b" 4) (.put ls2 "b" 5) (.put ls2 "b" 6) (.put ls2 "b" 7) (.put ls2 "b" 8) (is (= 8 (.get ls2 "b"))) ))) ================================================ FILE: storm-core/test/clj/backtype/storm/messaging_test.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.messaging-test (:use [clojure test]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) (deftest test-local-transport (doseq [transport-on? [false true]] (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 2 :daemon-conf {TOPOLOGY-WORKERS 2 STORM-LOCAL-MODE-ZMQ (if transport-on? true false) STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.zmq"}] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 2)} {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.) :parallelism-hint 6) }) results (complete-topology cluster topology ;; important for test that ;; #tuples = multiple of 4 and 6 :mock-sources {"1" [["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ]} )] (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]])) (read-tuples results "2"))))))) ================================================ FILE: storm-core/test/clj/backtype/storm/metrics_test.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.metrics-test (:use [clojure test]) (:import [backtype.storm.topology TopologyBuilder]) (:import [backtype.storm.generated InvalidTopologyException SubmitOptions TopologyInitialStatus]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestConfBolt AckFailMapTracker]) (:import [backtype.storm.metric.api CountMetric IMetricsConsumer$DataPoint IMetricsConsumer$TaskInfo]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) (:use [backtype.storm.metric testing])) (bootstrap) (defbolt acking-bolt {} {:prepare true} [conf context collector] (bolt (execute [tuple] (ack! collector tuple)))) (defbolt ack-every-other {} {:prepare true} [conf context collector] (let [state (atom -1)] (bolt (execute [tuple] (let [val (swap! state -)] (when (pos? val) (ack! collector tuple) )))))) (defn assert-loop [afn ids] (while (not (every? afn ids)) (Thread/sleep 1))) (defn assert-acked [tracker & ids] (assert-loop #(.isAcked tracker %) ids)) (defn assert-failed [tracker & ids] (assert-loop #(.isFailed tracker %) ids)) (defbolt count-acks {} {:prepare true} [conf context collector] (let [mycustommetric (CountMetric.)] (.registerMetric context "my-custom-metric" mycustommetric 5) (bolt (execute [tuple] (.incr mycustommetric) (ack! collector tuple))))) (def metrics-data backtype.storm.metric.testing/buffer) (defn wait-for-atleast-N-buckets! [N comp-id metric-name] (while (let [taskid->buckets (-> @metrics-data (get comp-id) (get metric-name))] (or (and (not= N 0) (nil? taskid->buckets)) (not-every? #(<= N %) (map (comp count second) taskid->buckets)))) ;; (println "Waiting for at least" N "timebuckets to appear in FakeMetricsConsumer for component id" comp-id ;; "and metric name" metric-name) (Thread/sleep 10))) (defn lookup-bucket-by-comp-id-&-metric-name! [comp-id metric-name] (-> @metrics-data (get comp-id) (get metric-name) (first) ;; pick first task in the list, ignore other tasks' metric data. (second) (or []))) (defmacro assert-buckets! [comp-id metric-name expected] `(do (let [N# (count ~expected)] (wait-for-atleast-N-buckets! N# ~comp-id ~metric-name) (is (= ~expected (subvec (lookup-bucket-by-comp-id-&-metric-name! ~comp-id ~metric-name) 0 N#)))))) (defmacro assert-metric-data-exists! [comp-id metric-name] `(is (not-empty (lookup-bucket-by-comp-id-&-metric-name! ~comp-id ~metric-name)))) (deftest test-custom-metric (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-METRICS-CONSUMER-REGISTER [{"class" "clojure.storm.metric.testing.FakeMetricConsumer"}]}] (let [feeder (feeder-spout ["field1"]) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec feeder)} {"2" (thrift/mk-bolt-spec {"1" :global} count-acks)})] (submit-local-topology (:nimbus cluster) "metrics-tester" {} topology) (.feed feeder ["a"] 1) (advance-cluster-time cluster 6) (assert-buckets! "2" "my-custom-metric" [1]) (advance-cluster-time cluster 5) (assert-buckets! "2" "my-custom-metric" [1 0]) (advance-cluster-time cluster 20) (assert-buckets! "2" "my-custom-metric" [1 0 0 0 0 0]) (.feed feeder ["b"] 2) (.feed feeder ["c"] 3) (advance-cluster-time cluster 5) (assert-buckets! "2" "my-custom-metric" [1 0 0 0 0 0 2])))) (deftest test-builtin-metrics-1 (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-METRICS-CONSUMER-REGISTER [{"class" "clojure.storm.metric.testing.FakeMetricConsumer"}] TOPOLOGY-STATS-SAMPLE-RATE 1.0 TOPOLOGY-BUILTIN-METRICS-BUCKET-SIZE-SECS 60}] (let [feeder (feeder-spout ["field1"]) topology (thrift/mk-topology {"myspout" (thrift/mk-spout-spec feeder)} {"mybolt" (thrift/mk-bolt-spec {"myspout" :shuffle} acking-bolt)})] (submit-local-topology (:nimbus cluster) "metrics-tester" {} topology) (.feed feeder ["a"] 1) (advance-cluster-time cluster 61) (assert-buckets! "myspout" "__ack-count/default" [1]) (assert-buckets! "myspout" "__emit-count/default" [1]) (assert-buckets! "myspout" "__transfer-count/default" [1]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [1]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [1]) (advance-cluster-time cluster 120) (assert-buckets! "myspout" "__ack-count/default" [1 0 0]) (assert-buckets! "myspout" "__emit-count/default" [1 0 0]) (assert-buckets! "myspout" "__transfer-count/default" [1 0 0]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [1 0 0]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [1 0 0]) (.feed feeder ["b"] 1) (.feed feeder ["c"] 1) (advance-cluster-time cluster 60) (assert-buckets! "myspout" "__ack-count/default" [1 0 0 2]) (assert-buckets! "myspout" "__emit-count/default" [1 0 0 2]) (assert-buckets! "myspout" "__transfer-count/default" [1 0 0 2]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [1 0 0 2]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [1 0 0 2])))) (deftest test-builtin-metrics-2 (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-METRICS-CONSUMER-REGISTER [{"class" "clojure.storm.metric.testing.FakeMetricConsumer"}] TOPOLOGY-STATS-SAMPLE-RATE 1.0 TOPOLOGY-BUILTIN-METRICS-BUCKET-SIZE-SECS 5}] (let [feeder (feeder-spout ["field1"]) tracker (AckFailMapTracker.) _ (.setAckFailDelegate feeder tracker) topology (thrift/mk-topology {"myspout" (thrift/mk-spout-spec feeder)} {"mybolt" (thrift/mk-bolt-spec {"myspout" :shuffle} ack-every-other)})] (submit-local-topology (:nimbus cluster) "metrics-tester" {} topology) (.feed feeder ["a"] 1) (advance-cluster-time cluster 6) (assert-acked tracker 1) (assert-buckets! "myspout" "__fail-count/default" []) (assert-buckets! "myspout" "__ack-count/default" [1]) (assert-buckets! "myspout" "__emit-count/default" [1]) (assert-buckets! "myspout" "__transfer-count/default" [1]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [1]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [1]) (.feed feeder ["b"] 2) (advance-cluster-time cluster 5) (assert-buckets! "myspout" "__fail-count/default" []) (assert-buckets! "myspout" "__ack-count/default" [1 0]) (assert-buckets! "myspout" "__emit-count/default" [1 1]) (assert-buckets! "myspout" "__transfer-count/default" [1 1]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [1 0]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [1 1]) (advance-cluster-time cluster 15) (assert-buckets! "myspout" "__ack-count/default" [1 0 0 0 0]) (assert-buckets! "myspout" "__emit-count/default" [1 1 0 0 0]) (assert-buckets! "myspout" "__transfer-count/default" [1 1 0 0 0]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [1 0 0 0 0]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [1 1 0 0 0]) (.feed feeder ["c"] 3) (advance-cluster-time cluster 15) (assert-buckets! "myspout" "__ack-count/default" [1 0 0 0 0 1 0 0]) (assert-buckets! "myspout" "__emit-count/default" [1 1 0 0 0 1 0 0]) (assert-buckets! "myspout" "__transfer-count/default" [1 1 0 0 0 1 0 0]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [1 0 0 0 0 1 0 0]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [1 1 0 0 0 1 0 0])))) (deftest test-builtin-metrics-3 (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-METRICS-CONSUMER-REGISTER [{"class" "clojure.storm.metric.testing.FakeMetricConsumer"}] TOPOLOGY-STATS-SAMPLE-RATE 1.0 TOPOLOGY-BUILTIN-METRICS-BUCKET-SIZE-SECS 5 TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true}] (let [feeder (feeder-spout ["field1"]) tracker (AckFailMapTracker.) _ (.setAckFailDelegate feeder tracker) topology (thrift/mk-topology {"myspout" (thrift/mk-spout-spec feeder)} {"mybolt" (thrift/mk-bolt-spec {"myspout" :global} ack-every-other)})] (submit-local-topology (:nimbus cluster) "timeout-tester" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10} topology) (.feed feeder ["a"] 1) (.feed feeder ["b"] 2) (.feed feeder ["c"] 3) (advance-cluster-time cluster 9) (assert-acked tracker 1 3) (assert-buckets! "myspout" "__ack-count/default" [2]) (assert-buckets! "myspout" "__emit-count/default" [3]) (assert-buckets! "myspout" "__transfer-count/default" [3]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [2]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [3]) (is (not (.isFailed tracker 2))) (advance-cluster-time cluster 30) (assert-failed tracker 2) (assert-buckets! "myspout" "__fail-count/default" [1]) (assert-buckets! "myspout" "__ack-count/default" [2 0 0 0]) (assert-buckets! "myspout" "__emit-count/default" [3 0 0 0]) (assert-buckets! "myspout" "__transfer-count/default" [3 0 0 0]) (assert-buckets! "mybolt" "__ack-count/myspout:default" [2 0 0 0]) (assert-buckets! "mybolt" "__execute-count/myspout:default" [3 0 0 0])))) (deftest test-system-bolt (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-METRICS-CONSUMER-REGISTER [{"class" "clojure.storm.metric.testing.FakeMetricConsumer"}] TOPOLOGY-BUILTIN-METRICS-BUCKET-SIZE-SECS 60}] (let [feeder (feeder-spout ["field1"]) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec feeder)} {})] (submit-local-topology (:nimbus cluster) "metrics-tester" {} topology) (.feed feeder ["a"] 1) (advance-cluster-time cluster 70) (assert-buckets! "__system" "newWorkerEvent" [1]) (assert-metric-data-exists! "__system" "uptimeSecs") (assert-metric-data-exists! "__system" "startTimeSecs") (advance-cluster-time cluster 180) (assert-buckets! "__system" "newWorkerEvent" [1 0 0 0]) ))) ================================================ FILE: storm-core/test/clj/backtype/storm/multilang_test.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.multilang-test (:use [clojure test]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) ;; (deftest test-multilang-fy ;; (with-local-cluster [cluster :supervisors 4] ;; (let [nimbus (:nimbus cluster) ;; topology (thrift/mk-topology ;; {"1" (thrift/mk-spout-spec (TestWordSpout. false))} ;; {"2" (thrift/mk-shell-bolt-spec {"1" :shuffle} "fancy" "tester.fy" ["word"] :parallelism-hint 1)} ;; )] ;; (submit-local-topology nimbus ;; "test" ;; {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true} ;; topology) ;; (Thread/sleep 10000) ;; (.killTopology nimbus "test") ;; (Thread/sleep 10000) ;; ))) (deftest test-multilang-rb (with-local-cluster [cluster :supervisors 4] (let [nimbus (:nimbus cluster) topology (thrift/mk-topology {"1" (thrift/mk-shell-spout-spec ["ruby" "tester_spout.rb"] ["word"])} {"2" (thrift/mk-shell-bolt-spec {"1" :shuffle} "ruby" "tester_bolt.rb" ["word"] :parallelism-hint 1)})] (submit-local-topology nimbus "test" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true} topology) (Thread/sleep 10000) (.killTopology nimbus "test") (Thread/sleep 10000)))) (deftest test-multilang-py (with-local-cluster [cluster :supervisors 4] (let [nimbus (:nimbus cluster) topology (thrift/mk-topology {"1" (thrift/mk-shell-spout-spec ["python" "tester_spout.py"] ["word"])} {"2" (thrift/mk-shell-bolt-spec {"1" :shuffle} ["python" "tester_bolt.py"] ["word"] :parallelism-hint 1)} )] (submit-local-topology nimbus "test" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true} topology) (Thread/sleep 10000) (.killTopology nimbus "test") (Thread/sleep 10000) ))) ================================================ FILE: storm-core/test/clj/backtype/storm/nimbus_test.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.nimbus-test (:use [clojure test]) (:require [backtype.storm.daemon [nimbus :as nimbus]]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter]) (:import [backtype.storm.scheduler INimbus]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) (defn storm-component->task-info [cluster storm-name] (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name) nimbus (:nimbus cluster)] (-> (.getUserTopology nimbus storm-id) (storm-task-info (from-json (.getTopologyConf nimbus storm-id))) reverse-map))) (defn storm-component->executor-info [cluster storm-name] (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name) nimbus (:nimbus cluster) storm-conf (from-json (.getTopologyConf nimbus storm-id)) topology (.getUserTopology nimbus storm-id) task->component (storm-task-info topology storm-conf) state (:storm-cluster-state cluster) get-component (comp task->component first)] (->> (.assignment-info state storm-id nil) :executor->node+port keys (map (fn [e] {e (get-component e)})) (apply merge) reverse-map))) (defn storm-num-workers [state storm-name] (let [storm-id (get-storm-id state storm-name) assignment (.assignment-info state storm-id nil)] (count (reverse-map (:executor->node+port assignment))) )) (defn topology-nodes [state storm-name] (let [storm-id (get-storm-id state storm-name) assignment (.assignment-info state storm-id nil)] (->> assignment :executor->node+port vals (map first) set ))) (defn topology-slots [state storm-name] (let [storm-id (get-storm-id state storm-name) assignment (.assignment-info state storm-id nil)] (->> assignment :executor->node+port vals set ))) (defn topology-node-distribution [state storm-name] (let [storm-id (get-storm-id state storm-name) assignment (.assignment-info state storm-id nil)] (->> assignment :executor->node+port vals set (group-by first) (map-val count) (map (fn [[_ amt]] {amt 1})) (apply merge-with +) ))) (defn topology-num-nodes [state storm-name] (count (topology-nodes state storm-name))) (defn executor-assignment [cluster storm-id executor-id] (let [state (:storm-cluster-state cluster) assignment (.assignment-info state storm-id nil)] ((:executor->node+port assignment) executor-id) )) (defn executor-start-times [cluster storm-id] (let [state (:storm-cluster-state cluster) assignment (.assignment-info state storm-id nil)] (:executor->start-time-secs assignment))) (defn do-executor-heartbeat [cluster storm-id executor] (let [state (:storm-cluster-state cluster) executor->node+port (:executor->node+port (.assignment-info state storm-id nil)) [node port] (get executor->node+port executor) curr-beat (.get-worker-heartbeat state storm-id node port) stats (:executor-stats curr-beat)] (.worker-heartbeat! state storm-id node port {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor nil})} ))) (defn slot-assignments [cluster storm-id] (let [state (:storm-cluster-state cluster) assignment (.assignment-info state storm-id nil)] (reverse-map (:executor->node+port assignment)) )) (defn task-ids [cluster storm-id] (let [nimbus (:nimbus cluster)] (-> (.getUserTopology nimbus storm-id) (storm-task-info (from-json (.getTopologyConf nimbus storm-id))) keys))) (defn topology-executors [cluster storm-id] (let [state (:storm-cluster-state cluster) assignment (.assignment-info state storm-id nil)] (keys (:executor->node+port assignment)) )) (defn check-distribution [items distribution] (let [dist (->> items (map count) multi-set)] (is (= dist (multi-set distribution))) )) (defn disjoint? [& sets] (let [combined (apply concat sets)] (= (count combined) (count (set combined))) )) (defnk check-consistency [cluster storm-name :assigned? true] (let [state (:storm-cluster-state cluster) storm-id (get-storm-id state storm-name) task-ids (task-ids cluster storm-id) assignment (.assignment-info state storm-id nil) executor->node+port (:executor->node+port assignment) task->node+port (to-task->node+port executor->node+port) assigned-task-ids (mapcat executor-id->tasks (keys executor->node+port)) all-nodes (set (map first (vals executor->node+port)))] (when assigned? (is (= (sort task-ids) (sort assigned-task-ids))) (doseq [t task-ids] (is (not-nil? (task->node+port t))))) (doseq [[e s] executor->node+port] (is (not-nil? s))) ;;(map str (-> (Thread/currentThread) .getStackTrace)) (is (= all-nodes (set (keys (:node->host assignment))))) (doseq [[e s] executor->node+port] (is (not-nil? ((:executor->start-time-secs assignment) e)))) )) (deftest test-bogusId (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] (let [state (:storm-cluster-state cluster) nimbus (:nimbus cluster)] (is (thrown? NotAliveException (.getTopologyConf nimbus "bogus-id"))) (is (thrown? NotAliveException (.getTopology nimbus "bogus-id"))) (is (thrown? NotAliveException (.getUserTopology nimbus "bogus-id"))) (is (thrown? NotAliveException (.getTopologyInfo nimbus "bogus-id"))) ))) (deftest test-assignment (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] (let [state (:storm-cluster-state cluster) nimbus (:nimbus cluster) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. false) :parallelism-hint 3)} {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 4) "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.))}) topology2 (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 12)} {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 6) "3" (thrift/mk-bolt-spec {"1" :global} (TestPlannerBolt.) :parallelism-hint 8) "4" (thrift/mk-bolt-spec {"1" :global "2" :none} (TestPlannerBolt.) :parallelism-hint 4)} ) _ (submit-local-topology nimbus "mystorm" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 4} topology) task-info (storm-component->task-info cluster "mystorm")] (check-consistency cluster "mystorm") ;; 3 should be assigned once (if it were optimized, we'd have ;; different topology) (is (= 1 (count (.assignments state nil)))) (is (= 1 (count (task-info "1")))) (is (= 4 (count (task-info "2")))) (is (= 1 (count (task-info "3")))) (is (= 4 (storm-num-workers state "mystorm"))) (submit-local-topology nimbus "storm2" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20} topology2) (check-consistency cluster "storm2") (is (= 2 (count (.assignments state nil)))) (let [task-info (storm-component->task-info cluster "storm2")] (is (= 12 (count (task-info "1")))) (is (= 6 (count (task-info "2")))) (is (= 8 (count (task-info "3")))) (is (= 4 (count (task-info "4")))) (is (= 8 (storm-num-workers state "storm2"))) ) ))) (defn isolation-nimbus [] (let [standalone (nimbus/standalone-nimbus)] (reify INimbus (prepare [this conf local-dir] (.prepare standalone conf local-dir) ) (allSlotsAvailableForScheduling [this supervisors topologies topologies-missing-assignments] (.allSlotsAvailableForScheduling standalone supervisors topologies topologies-missing-assignments)) (assignSlots [this topology slots] (.assignSlots standalone topology slots) ) (getForcedScheduler [this] (.getForcedScheduler standalone)) (getHostName [this supervisors node-id] node-id )))) (deftest test-isolated-assignment (with-simulated-time-local-cluster [cluster :supervisors 6 :ports-per-supervisor 3 :inimbus (isolation-nimbus) :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 STORM-SCHEDULER "backtype.storm.scheduler.IsolationScheduler" ISOLATION-SCHEDULER-MACHINES {"tester1" 3 "tester2" 2} NIMBUS-MONITOR-FREQ-SECS 10 }] (letlocals (bind state (:storm-cluster-state cluster)) (bind nimbus (:nimbus cluster)) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. false) :parallelism-hint 3)} {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 5) "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.))})) (submit-local-topology nimbus "noniso" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 4} topology) (advance-cluster-time cluster 1) (is (= 4 (topology-num-nodes state "noniso"))) (is (= 4 (storm-num-workers state "noniso"))) (submit-local-topology nimbus "tester1" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 6} topology) (submit-local-topology nimbus "tester2" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 6} topology) (advance-cluster-time cluster 1) (bind task-info-tester1 (storm-component->task-info cluster "tester1")) (bind task-info-tester2 (storm-component->task-info cluster "tester2")) (is (= 1 (topology-num-nodes state "noniso"))) (is (= 3 (storm-num-workers state "noniso"))) (is (= {2 3} (topology-node-distribution state "tester1"))) (is (= {3 2} (topology-node-distribution state "tester2"))) (is (apply disjoint? (map (partial topology-nodes state) ["noniso" "tester1" "tester2"]))) (check-consistency cluster "tester1") (check-consistency cluster "tester2") (check-consistency cluster "noniso") ;;check that nothing gets reassigned (bind tester1-slots (topology-slots state "tester1")) (bind tester2-slots (topology-slots state "tester2")) (bind noniso-slots (topology-slots state "noniso")) (advance-cluster-time cluster 20) (is (= tester1-slots (topology-slots state "tester1"))) (is (= tester2-slots (topology-slots state "tester2"))) (is (= noniso-slots (topology-slots state "noniso"))) ))) (deftest test-zero-executor-or-tasks (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] (let [state (:storm-cluster-state cluster) nimbus (:nimbus cluster) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. false) :parallelism-hint 3 :conf {TOPOLOGY-TASKS 0})} {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 1 :conf {TOPOLOGY-TASKS 2}) "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.) :conf {TOPOLOGY-TASKS 5})}) _ (submit-local-topology nimbus "mystorm" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 4} topology) task-info (storm-component->task-info cluster "mystorm")] (check-consistency cluster "mystorm") (is (= 0 (count (task-info "1")))) (is (= 2 (count (task-info "2")))) (is (= 5 (count (task-info "3")))) (is (= 2 (storm-num-workers state "mystorm"))) ;; because only 2 executors ))) (deftest test-executor-assignments (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] (let [nimbus (:nimbus cluster) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3 :conf {TOPOLOGY-TASKS 5})} {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 8 :conf {TOPOLOGY-TASKS 2}) "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.) :parallelism-hint 3)}) _ (submit-local-topology nimbus "mystorm" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 4} topology) task-info (storm-component->task-info cluster "mystorm") executor-info (->> (storm-component->executor-info cluster "mystorm") (map-val #(map executor-id->tasks %)))] (check-consistency cluster "mystorm") (is (= 5 (count (task-info "1")))) (check-distribution (executor-info "1") [2 2 1]) (is (= 2 (count (task-info "2")))) (check-distribution (executor-info "2") [1 1]) (is (= 3 (count (task-info "3")))) (check-distribution (executor-info "3") [1 1 1]) ))) (deftest test-over-parallelism-assignment (with-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] (let [state (:storm-cluster-state cluster) nimbus (:nimbus cluster) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 21)} {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 9) "3" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 2) "4" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 10)} ) _ (submit-local-topology nimbus "test" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 7} topology) task-info (storm-component->task-info cluster "test")] (check-consistency cluster "test") (is (= 21 (count (task-info "1")))) (is (= 9 (count (task-info "2")))) (is (= 2 (count (task-info "3")))) (is (= 10 (count (task-info "4")))) (is (= 7 (storm-num-workers state "test"))) ))) (deftest test-kill-storm (with-simulated-time-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5 :daemon-conf {SUPERVISOR-ENABLE false NIMBUS-TASK-TIMEOUT-SECS 30 NIMBUS-MONITOR-FREQ-SECS 10 TOPOLOGY-ACKER-EXECUTORS 0}] (letlocals (bind conf (:daemon-conf cluster)) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 14)} {} )) (bind state (:storm-cluster-state cluster)) (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20} topology) (bind storm-id (get-storm-id state "test")) (advance-cluster-time cluster 5) (is (not-nil? (.storm-base state storm-id nil))) (is (not-nil? (.assignment-info state storm-id nil))) (.killTopology (:nimbus cluster) "test") ;; check that storm is deactivated but alive (is (= :killed (-> (.storm-base state storm-id nil) :status :type))) (is (not-nil? (.assignment-info state storm-id nil))) (advance-cluster-time cluster 18) ;; check that storm is deactivated but alive (is (= 1 (count (.heartbeat-storms state)))) (advance-cluster-time cluster 3) (is (nil? (.storm-base state storm-id nil))) (is (nil? (.assignment-info state storm-id nil))) ;; cleanup happens on monitoring thread (advance-cluster-time cluster 11) (is (empty? (.heartbeat-storms state))) ;; TODO: check that code on nimbus was cleaned up locally... (is (thrown? NotAliveException (.killTopology (:nimbus cluster) "lalala"))) (submit-local-topology (:nimbus cluster) "2test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10} topology) (is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology))) (bind storm-id (get-storm-id state "2test")) (is (not-nil? (.storm-base state storm-id nil))) (.killTopology (:nimbus cluster) "2test") (is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology))) (advance-cluster-time cluster 5) (is (= 1 (count (.heartbeat-storms state)))) (advance-cluster-time cluster 6) (is (nil? (.storm-base state storm-id nil))) (is (nil? (.assignment-info state storm-id nil))) (advance-cluster-time cluster 11) (is (= 0 (count (.heartbeat-storms state)))) (submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology) (bind storm-id3 (get-storm-id state "test3")) (advance-cluster-time cluster 1) (.remove-storm! state storm-id3) (is (nil? (.storm-base state storm-id3 nil))) (is (nil? (.assignment-info state storm-id3 nil))) (advance-cluster-time cluster 11) (is (= 0 (count (.heartbeat-storms state)))) ;; this guarantees that monitor thread won't trigger for 10 more seconds (advance-time-secs! 11) (wait-until-cluster-waiting cluster) (submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology) (bind storm-id3 (get-storm-id state "test3")) (bind executor-id (first (topology-executors cluster storm-id3))) (do-executor-heartbeat cluster storm-id3 executor-id) (.killTopology (:nimbus cluster) "test3") (advance-cluster-time cluster 6) (is (= 1 (count (.heartbeat-storms state)))) (advance-cluster-time cluster 5) (is (= 0 (count (.heartbeat-storms state)))) ;; test kill with opts (submit-local-topology (:nimbus cluster) "test4" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 100} topology) (.killTopologyWithOpts (:nimbus cluster) "test4" (doto (KillOptions.) (.set_wait_secs 10))) (bind storm-id4 (get-storm-id state "test4")) (advance-cluster-time cluster 9) (is (not-nil? (.assignment-info state storm-id4 nil))) (advance-cluster-time cluster 2) (is (nil? (.assignment-info state storm-id4 nil))) ))) (deftest test-reassignment (with-simulated-time-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5 :daemon-conf {SUPERVISOR-ENABLE false NIMBUS-TASK-LAUNCH-SECS 60 NIMBUS-TASK-TIMEOUT-SECS 20 NIMBUS-MONITOR-FREQ-SECS 10 NIMBUS-SUPERVISOR-TIMEOUT-SECS 100 TOPOLOGY-ACKER-EXECUTORS 0}] (letlocals (bind conf (:daemon-conf cluster)) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)} {} )) (bind state (:storm-cluster-state cluster)) (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 2} topology) (check-consistency cluster "test") (bind storm-id (get-storm-id state "test")) (bind [executor-id1 executor-id2] (topology-executors cluster storm-id)) (bind ass1 (executor-assignment cluster storm-id executor-id1)) (bind ass2 (executor-assignment cluster storm-id executor-id2)) (advance-cluster-time cluster 59) (do-executor-heartbeat cluster storm-id executor-id1) (do-executor-heartbeat cluster storm-id executor-id2) (advance-cluster-time cluster 13) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (is (= ass2 (executor-assignment cluster storm-id executor-id2))) (do-executor-heartbeat cluster storm-id executor-id1) (advance-cluster-time cluster 11) (do-executor-heartbeat cluster storm-id executor-id1) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (check-consistency cluster "test") ; have to wait an extra 10 seconds because nimbus may not ; resynchronize its heartbeat time till monitor-time secs after (advance-cluster-time cluster 11) (do-executor-heartbeat cluster storm-id executor-id1) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (check-consistency cluster "test") (advance-cluster-time cluster 11) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (is (not= ass2 (executor-assignment cluster storm-id executor-id2))) (bind ass2 (executor-assignment cluster storm-id executor-id2)) (check-consistency cluster "test") (advance-cluster-time cluster 31) (is (not= ass1 (executor-assignment cluster storm-id executor-id1))) (is (= ass2 (executor-assignment cluster storm-id executor-id2))) ; tests launch timeout (check-consistency cluster "test") (bind ass1 (executor-assignment cluster storm-id executor-id1)) (bind active-supervisor (first ass2)) (kill-supervisor cluster active-supervisor) (doseq [i (range 12)] (do-executor-heartbeat cluster storm-id executor-id1) (do-executor-heartbeat cluster storm-id executor-id2) (advance-cluster-time cluster 10) ) ;; tests that it doesn't reassign executors if they're heartbeating even if supervisor times out (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (is (= ass2 (executor-assignment cluster storm-id executor-id2))) (check-consistency cluster "test") (advance-cluster-time cluster 30) (bind ass1 (executor-assignment cluster storm-id executor-id1)) (bind ass2 (executor-assignment cluster storm-id executor-id2)) (is (not-nil? ass1)) (is (not-nil? ass2)) (is (not= active-supervisor (first (executor-assignment cluster storm-id executor-id2)))) (is (not= active-supervisor (first (executor-assignment cluster storm-id executor-id1)))) (check-consistency cluster "test") (doseq [supervisor-id (.supervisors state nil)] (kill-supervisor cluster supervisor-id)) (advance-cluster-time cluster 90) (bind ass1 (executor-assignment cluster storm-id executor-id1)) (bind ass2 (executor-assignment cluster storm-id executor-id2)) (is (nil? ass1)) (is (nil? ass2)) (check-consistency cluster "test" :assigned? false) (add-supervisor cluster) (advance-cluster-time cluster 11) (check-consistency cluster "test") ))) (deftest test-reassignment-to-constrained-cluster (with-simulated-time-local-cluster [cluster :supervisors 0 :daemon-conf {SUPERVISOR-ENABLE false NIMBUS-TASK-LAUNCH-SECS 60 NIMBUS-TASK-TIMEOUT-SECS 20 NIMBUS-MONITOR-FREQ-SECS 10 NIMBUS-SUPERVISOR-TIMEOUT-SECS 100 TOPOLOGY-ACKER-EXECUTORS 0}] (letlocals (add-supervisor cluster :ports 1 :id "a") (add-supervisor cluster :ports 1 :id "b") (bind conf (:daemon-conf cluster)) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)} {} )) (bind state (:storm-cluster-state cluster)) (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 2} topology) (check-consistency cluster "test") (bind storm-id (get-storm-id state "test")) (bind [executor-id1 executor-id2] (topology-executors cluster storm-id)) (bind ass1 (executor-assignment cluster storm-id executor-id1)) (bind ass2 (executor-assignment cluster storm-id executor-id2)) (advance-cluster-time cluster 59) (do-executor-heartbeat cluster storm-id executor-id1) (do-executor-heartbeat cluster storm-id executor-id2) (advance-cluster-time cluster 13) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (is (= ass2 (executor-assignment cluster storm-id executor-id2))) (kill-supervisor cluster "b") (do-executor-heartbeat cluster storm-id executor-id1) (advance-cluster-time cluster 11) (do-executor-heartbeat cluster storm-id executor-id1) (advance-cluster-time cluster 11) (do-executor-heartbeat cluster storm-id executor-id1) (advance-cluster-time cluster 11) (do-executor-heartbeat cluster storm-id executor-id1) (advance-cluster-time cluster 11) (do-executor-heartbeat cluster storm-id executor-id1) (check-consistency cluster "test") (is (= 1 (storm-num-workers state "test"))) ))) (defn check-executor-distribution [slot-executors distribution] (check-distribution (vals slot-executors) distribution)) (defn check-num-nodes [slot-executors num-nodes] (let [nodes (->> slot-executors keys (map first) set)] (is (= num-nodes (count nodes))) )) (deftest test-reassign-squeezed-topology (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 1 :daemon-conf {SUPERVISOR-ENABLE false NIMBUS-TASK-LAUNCH-SECS 60 NIMBUS-TASK-TIMEOUT-SECS 20 NIMBUS-MONITOR-FREQ-SECS 10 TOPOLOGY-ACKER-EXECUTORS 0}] (letlocals (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 9)} {})) (bind state (:storm-cluster-state cluster)) (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 4} topology) ; distribution should be 2, 2, 2, 3 ideally (bind storm-id (get-storm-id state "test")) (bind slot-executors (slot-assignments cluster storm-id)) (check-executor-distribution slot-executors [9]) (check-consistency cluster "test") (add-supervisor cluster :ports 2) (advance-cluster-time cluster 11) (bind slot-executors (slot-assignments cluster storm-id)) (bind executor->start (executor-start-times cluster storm-id)) (check-executor-distribution slot-executors [3 3 3]) (check-consistency cluster "test") (add-supervisor cluster :ports 8) ;; this actually works for any time > 0, since zookeeper fires an event causing immediate reassignment ;; doesn't work for time = 0 because it's not waiting for cluster yet, so test might happen before reassignment finishes (advance-cluster-time cluster 11) (bind slot-executors2 (slot-assignments cluster storm-id)) (bind executor->start2 (executor-start-times cluster storm-id)) (check-executor-distribution slot-executors2 [2 2 2 3]) (check-consistency cluster "test") (bind common (first (find-first (fn [[k v]] (= 3 (count v))) slot-executors2))) (is (not-nil? common)) (is (= (slot-executors2 common) (slot-executors common))) ;; check that start times are changed for everything but the common one (bind same-executors (slot-executors2 common)) (bind changed-executors (apply concat (vals (dissoc slot-executors2 common)))) (doseq [t same-executors] (is (= (executor->start t) (executor->start2 t)))) (doseq [t changed-executors] (is (not= (executor->start t) (executor->start2 t)))) ))) (deftest test-rebalance (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false NIMBUS-MONITOR-FREQ-SECS 10 TOPOLOGY-MESSAGE-TIMEOUT-SECS 30 TOPOLOGY-ACKER-EXECUTORS 0}] (letlocals (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} {})) (bind state (:storm-cluster-state cluster)) (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 60} topology) (bind storm-id (get-storm-id state "test")) (add-supervisor cluster :ports 3) (add-supervisor cluster :ports 3) (advance-cluster-time cluster 91) (bind slot-executors (slot-assignments cluster storm-id)) ;; check that all workers are on one machine (check-executor-distribution slot-executors [1 1 1]) (check-num-nodes slot-executors 1) (.rebalance (:nimbus cluster) "test" (RebalanceOptions.)) (advance-cluster-time cluster 31) (check-executor-distribution slot-executors [1 1 1]) (check-num-nodes slot-executors 1) (advance-cluster-time cluster 30) (bind slot-executors (slot-assignments cluster storm-id)) (check-executor-distribution slot-executors [1 1 1]) (check-num-nodes slot-executors 3) (is (thrown? InvalidTopologyException (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.) (.set_num_executors {"1" 0}) )))) ))) (deftest test-rebalance-change-parallelism (with-simulated-time-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false NIMBUS-MONITOR-FREQ-SECS 10 TOPOLOGY-ACKER-EXECUTORS 0}] (letlocals (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 6 :conf {TOPOLOGY-TASKS 12})} {})) (bind state (:storm-cluster-state cluster)) (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 30} topology) (bind storm-id (get-storm-id state "test")) (bind checker (fn [distribution] (check-executor-distribution (slot-assignments cluster storm-id) distribution))) (checker [2 2 2]) (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.) (.set_num_workers 6) )) (advance-cluster-time cluster 29) (checker [2 2 2]) (advance-cluster-time cluster 3) (checker [1 1 1 1 1 1]) (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.) (.set_num_executors {"1" 1}) )) (advance-cluster-time cluster 29) (checker [1 1 1 1 1 1]) (advance-cluster-time cluster 3) (checker [1]) (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.) (.set_num_executors {"1" 8}) (.set_num_workers 4) )) (advance-cluster-time cluster 32) (checker [2 2 2 2]) (check-consistency cluster "test") (bind executor-info (->> (storm-component->executor-info cluster "test") (map-val #(map executor-id->tasks %)))) (check-distribution (executor-info "1") [2 2 2 2 1 1 1 1]) ))) (deftest test-submit-invalid (with-simulated-time-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] (letlocals (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 0 :conf {TOPOLOGY-TASKS 1})} {})) (is (thrown? InvalidTopologyException (submit-local-topology (:nimbus cluster) "test" {} topology))) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 1 :conf {TOPOLOGY-TASKS 1})} {})) (is (thrown? InvalidTopologyException (submit-local-topology (:nimbus cluster) "test/aaa" {} topology))) ))) (deftest test-cleans-corrupt (with-inprocess-zookeeper zk-port (with-local-tmp [nimbus-dir] (letlocals (bind conf (merge (read-storm-config) {STORM-ZOOKEEPER-SERVERS ["localhost"] STORM-CLUSTER-MODE "local" STORM-ZOOKEEPER-PORT zk-port STORM-LOCAL-DIR nimbus-dir})) (bind cluster-state (cluster/mk-storm-cluster-state conf)) (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus))) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} {})) (submit-local-topology nimbus "t1" {} topology) (submit-local-topology nimbus "t2" {} topology) (bind storm-id1 (get-storm-id cluster-state "t1")) (bind storm-id2 (get-storm-id cluster-state "t2")) (.shutdown nimbus) (rmr (master-stormdist-root conf storm-id1)) (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus))) (is ( = #{storm-id2} (set (.active-storms cluster-state)))) (.shutdown nimbus) (.disconnect cluster-state) )))) (deftest test-no-overlapping-slots ;; test that same node+port never appears across 2 assignments ) (deftest test-stateless ;; test that nimbus can die and restart without any problems ) (deftest test-clean-inbox "Tests that the inbox correctly cleans jar files." (with-simulated-time (with-local-tmp [dir-location] (let [dir (File. dir-location) mk-file (fn [name seconds-ago] (let [f (File. (str dir-location "/" name)) t (- (Time/currentTimeMillis) (* seconds-ago 1000))] (FileUtils/touch f) (.setLastModified f t))) assert-files-in-dir (fn [compare-file-names] (let [file-names (map #(.getName %) (file-seq dir))] (is (= (sort compare-file-names) (sort (filter #(.endsWith % ".jar") file-names)) ))))] ;; Make three files a.jar, b.jar, c.jar. ;; a and b are older than c and should be deleted first. (advance-time-secs! 100) (doseq [fs [["a.jar" 20] ["b.jar" 20] ["c.jar" 0]]] (apply mk-file fs)) (assert-files-in-dir ["a.jar" "b.jar" "c.jar"]) (nimbus/clean-inbox dir-location 10) (assert-files-in-dir ["c.jar"]) ;; Cleanit again, c.jar should stay (advance-time-secs! 5) (nimbus/clean-inbox dir-location 10) (assert-files-in-dir ["c.jar"]) ;; Advance time, clean again, c.jar should be deleted. (advance-time-secs! 5) (nimbus/clean-inbox dir-location 10) (assert-files-in-dir []) )))) ================================================ FILE: storm-core/test/clj/backtype/storm/scheduler_test.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.scheduler-test (:use [clojure test]) (:use [backtype.storm bootstrap config testing]) (:require [backtype.storm.daemon [nimbus :as nimbus]]) (:import [backtype.storm.generated StormTopology]) (:import [backtype.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails SchedulerAssignmentImpl Topologies TopologyDetails])) (bootstrap) (defn clojurify-executor->slot [executorToSlot] (into {} (for [[executor slot] executorToSlot] {[(.getStartTask executor) (.getEndTask executor)] [(.getNodeId slot) (.getPort slot)]}))) (defn clojurify-executor->comp [executorToComp] (into {} (for [[executor component] executorToComp] {[(.getStartTask executor) (.getEndTask executor)] component}))) (defn clojurify-component->executors [compToExecutor] (into {} (for [[component executors] compToExecutor :let [new-executors (set (map #(vector (.getStartTask %) (.getEndTask %)) executors))]] {component new-executors}))) (deftest test-supervisor-details (let [executor->slot {(ExecutorDetails. (int 1) (int 5)) (WorkerSlot. "supervisor1" (int 1)) (ExecutorDetails. (int 6) (int 10)) (WorkerSlot. "supervisor2" (int 2))} topology-id "topology1" assignment (SchedulerAssignmentImpl. topology-id executor->slot)] ;; test assign (.assign assignment (WorkerSlot. "supervisor1" 1) (list (ExecutorDetails. (int 11) (int 15)) (ExecutorDetails. (int 16) (int 20)))) (is (= {[1 5] ["supervisor1" 1] [6 10] ["supervisor2" 2] [11 15] ["supervisor1" 1] [16 20] ["supervisor1" 1]} (clojurify-executor->slot (.getExecutorToSlot assignment)))) ;; test isSlotOccupied (is (= true (.isSlotOccupied assignment (WorkerSlot. "supervisor2" (int 2))))) (is (= true (.isSlotOccupied assignment (WorkerSlot. "supervisor1" (int 1))))) ;; test isExecutorAssigned (is (= true (.isExecutorAssigned assignment (ExecutorDetails. (int 1) (int 5))))) (is (= false (.isExecutorAssigned assignment (ExecutorDetails. (int 21) (int 25))))) ;; test unassignBySlot (.unassignBySlot assignment (WorkerSlot. "supervisor1" (int 1))) (is (= {[6 10] ["supervisor2" 2]} (clojurify-executor->slot (.getExecutorToSlot assignment)))) )) (deftest test-topologies (let [executor1 (ExecutorDetails. (int 1) (int 5)) executor2 (ExecutorDetails. (int 6) (int 10)) topology1 (TopologyDetails. "topology1" {TOPOLOGY-NAME "topology-name-1"} (StormTopology.) 1 {executor1 "spout1" executor2 "bolt1"}) ;; test topology.selectExecutorToComponent executor->comp (.selectExecutorToComponent topology1 (list executor1)) _ (is (= (clojurify-executor->comp {executor1 "spout1"}) (clojurify-executor->comp executor->comp))) ;; test topologies.getById topology2 (TopologyDetails. "topology2" {TOPOLOGY-NAME "topology-name-2"} (StormTopology.) 1 {}) topologies (Topologies. {"topology1" topology1 "topology2" topology2}) _ (is (= "topology1" (->> "topology1" (.getById topologies) .getId))) ;; test topologies.getByName _ (is (= "topology2" (->> "topology-name-2" (.getByName topologies) .getId))) ] ) ) (deftest test-cluster (let [supervisor1 (SupervisorDetails. "supervisor1" "192.168.0.1" (list ) (map int (list 1 3 5 7 9))) supervisor2 (SupervisorDetails. "supervisor2" "192.168.0.2" (list ) (map int (list 2 4 6 8 10))) executor1 (ExecutorDetails. (int 1) (int 5)) executor2 (ExecutorDetails. (int 6) (int 10)) executor3 (ExecutorDetails. (int 11) (int 15)) executor11 (ExecutorDetails. (int 100) (int 105)) executor12 (ExecutorDetails. (int 106) (int 110)) executor21 (ExecutorDetails. (int 201) (int 205)) executor22 (ExecutorDetails. (int 206) (int 210)) ;; topology1 needs scheduling: executor3 is NOT assigned a slot. topology1 (TopologyDetails. "topology1" {TOPOLOGY-NAME "topology-name-1"} (StormTopology.) 2 {executor1 "spout1" executor2 "bolt1" executor3 "bolt2"}) ;; topology2 is fully scheduled topology2 (TopologyDetails. "topology2" {TOPOLOGY-NAME "topology-name-2"} (StormTopology.) 2 {executor11 "spout11" executor12 "bolt12"}) ;; topology3 needs scheduling, since the assignment is squeezed topology3 (TopologyDetails. "topology3" {TOPOLOGY-NAME "topology-name-3"} (StormTopology.) 2 {executor21 "spout21" executor22 "bolt22"}) topologies (Topologies. {"topology1" topology1 "topology2" topology2 "topology3" topology3}) executor->slot1 {executor1 (WorkerSlot. "supervisor1" (int 1)) executor2 (WorkerSlot. "supervisor2" (int 2))} executor->slot2 {executor11 (WorkerSlot. "supervisor1" (int 3)) executor12 (WorkerSlot. "supervisor2" (int 4))} executor->slot3 {executor21 (WorkerSlot. "supervisor1" (int 5)) executor22 (WorkerSlot. "supervisor1" (int 5))} assignment1 (SchedulerAssignmentImpl. "topology1" executor->slot1) assignment2 (SchedulerAssignmentImpl. "topology2" executor->slot2) assignment3 (SchedulerAssignmentImpl. "topology3" executor->slot3) cluster (Cluster. (nimbus/standalone-nimbus) {"supervisor1" supervisor1 "supervisor2" supervisor2} {"topology1" assignment1 "topology2" assignment2 "topology3" assignment3})] ;; test Cluster constructor (is (= #{"supervisor1" "supervisor2"} (->> cluster .getSupervisors keys set))) (is (= #{"topology1" "topology2" "topology3"} (->> cluster .getAssignments keys set))) ;; test Cluster.getUnassignedExecutors (is (= (set (list executor3)) (-> cluster (.getUnassignedExecutors topology1) set))) (is (= true (empty? (-> cluster (.getUnassignedExecutors topology2))))) ;; test Cluster.needsScheduling (is (= true (.needsScheduling cluster topology1))) (is (= false (.needsScheduling cluster topology2))) (is (= true (.needsScheduling cluster topology3))) ;; test Cluster.needsSchedulingTopologies (is (= #{"topology1" "topology3"} (->> (.needsSchedulingTopologies cluster topologies) (map (fn [topology] (.getId topology))) set))) ;; test Cluster.getNeedsSchedulingExecutorToComponents (is (= {executor3 "bolt2"} (.getNeedsSchedulingExecutorToComponents cluster topology1))) (is (= true (empty? (.getNeedsSchedulingExecutorToComponents cluster topology2)))) (is (= true (empty? (.getNeedsSchedulingExecutorToComponents cluster topology3)))) ;; test Cluster.getNeedsSchedulingComponentToExecutors (is (= {"bolt2" #{[(.getStartTask executor3) (.getEndTask executor3)]}} (clojurify-component->executors (.getNeedsSchedulingComponentToExecutors cluster topology1)))) (is (= true (empty? (.getNeedsSchedulingComponentToExecutors cluster topology2)))) (is (= true (empty? (.getNeedsSchedulingComponentToExecutors cluster topology3)))) ;; test Cluster.getUsedPorts (is (= #{1 3 5} (set (.getUsedPorts cluster supervisor1)))) (is (= #{2 4} (set (.getUsedPorts cluster supervisor2)))) (is (= #{1 3 5} (set (.getUsedPorts cluster supervisor1)))) ;; test Cluster.getAvailablePorts (is (= #{7 9} (set (.getAvailablePorts cluster supervisor1)))) (is (= #{6 8 10} (set (.getAvailablePorts cluster supervisor2)))) ;; test Cluster.getAvailableSlots (is (= #{["supervisor1" 7] ["supervisor1" 9]} (set (map (fn [slot] [(.getNodeId slot) (.getPort slot)]) (.getAvailableSlots cluster supervisor1))))) (is (= #{["supervisor2" 6] ["supervisor2" 8] ["supervisor2" 10]} (set (map (fn [slot] [(.getNodeId slot) (.getPort slot)]) (.getAvailableSlots cluster supervisor2))))) ;; test Cluster.getAvailableSlots (is (= #{["supervisor1" 7] ["supervisor1" 9] ["supervisor2" 6] ["supervisor2" 8] ["supervisor2" 10]} (set (map (fn [slot] [(.getNodeId slot) (.getPort slot)]) (.getAvailableSlots cluster))))) ;; test Cluster.getAssignedNumWorkers (is (= 2 (.getAssignedNumWorkers cluster topology1))) (is (= 2 (.getAssignedNumWorkers cluster topology2))) (is (= 1 (.getAssignedNumWorkers cluster topology3))) ;; test Cluster.isSlotOccupied (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 1))))) (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 3))))) (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 5))))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 7))))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 9))))) (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor2" (int 2))))) (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor2" (int 4))))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor2" (int 6))))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor2" (int 8))))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor2" (int 10))))) ;; test Cluster.getAssignmentById (is (= assignment1 (.getAssignmentById cluster "topology1"))) (is (= assignment2 (.getAssignmentById cluster "topology2"))) (is (= assignment3 (.getAssignmentById cluster "topology3"))) ;; test Cluster.getSupervisorById (is (= supervisor1 (.getSupervisorById cluster "supervisor1"))) (is (= supervisor2 (.getSupervisorById cluster "supervisor2"))) ;; test Cluster.getSupervisorsByHost (is (= #{supervisor1} (set (.getSupervisorsByHost cluster "192.168.0.1")))) (is (= #{supervisor2} (set (.getSupervisorsByHost cluster "192.168.0.2")))) ;; ==== the following tests will change the state of the cluster, so put it here at the end ==== ;; test Cluster.assign (.assign cluster (WorkerSlot. "supervisor1" (int 7)) "topology1" (list executor3)) (is (= false (.needsScheduling cluster topology1))) (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 7))))) ;; revert the change (.freeSlot cluster (WorkerSlot. "supervisor1" (int 7))) ;; test Cluster.assign: if a executor is already assigned, there will be an exception (let [has-exception (try (.assign cluster (WorkerSlot. "supervisor1" (int 9)) "topology1" (list executor1)) false (catch Exception e true))] (is (= true has-exception))) ;; test Cluster.assign: if a slot is occupied, there will be an exception (let [has-exception (try (.assign cluster (WorkerSlot. "supervisor2" (int 4)) "topology1" (list executor3)) false (catch Exception e true))] (is (= true has-exception))) ;; test Cluster.freeSlot (.freeSlot cluster (WorkerSlot. "supervisor1" (int 7))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 7))))) ;; test Cluster.freeSlots (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 1))))) (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 3))))) (is (= true (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 5))))) (.freeSlots cluster (list (WorkerSlot. "supervisor1" (int 1)) (WorkerSlot. "supervisor1" (int 3)) (WorkerSlot. "supervisor1" (int 5)))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 1))))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 3))))) (is (= false (.isSlotOccupied cluster (WorkerSlot. "supervisor1" (int 5))))) )) ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.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.security.auth.AuthUtils-test (:import [backtype.storm.security.auth AuthUtils]) (:import [java.io IOException]) (:import [javax.security.auth.login AppConfigurationEntry Configuration]) (:import [org.mockito Mockito]) (:use [clojure test]) ) (deftest test-throws-on-missing-section (is (thrown? IOException (AuthUtils/get (Mockito/mock Configuration) "bogus-section" ""))) ) (defn- mk-mock-app-config-entry [] (let [toRet (Mockito/mock AppConfigurationEntry)] (. (Mockito/when (.getOptions toRet)) thenReturn (hash-map)) toRet ) ) (deftest test-returns-null-if-no-such-section (let [entry (mk-mock-app-config-entry) entries (into-array (.getClass entry) [entry]) section "bogus-section" conf (Mockito/mock Configuration)] (. (Mockito/when (. conf getAppConfigurationEntry section )) thenReturn entries) (is (nil? (AuthUtils/get conf section "nonexistent-key"))) ) ) (deftest test-returns-first-value-for-valid-key (let [k "the-key" expected "good-value" empty-entry (mk-mock-app-config-entry) bad-entry (Mockito/mock AppConfigurationEntry) good-entry (Mockito/mock AppConfigurationEntry) conf (Mockito/mock Configuration)] (. (Mockito/when (.getOptions bad-entry)) thenReturn {k "bad-value"}) (. (Mockito/when (.getOptions good-entry)) thenReturn {k expected}) (let [entries (into-array (.getClass empty-entry) [empty-entry good-entry bad-entry]) section "bogus-section"] (. (Mockito/when (. conf getAppConfigurationEntry section)) thenReturn entries) (is (not (nil? (AuthUtils/get conf section k)))) (is (= (AuthUtils/get conf section k) expected)) ) ) ) ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/ReqContext_test.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.security.auth.ReqContext-test (:import [backtype.storm.security.auth ReqContext]) (:import [java.net InetAddress]) (:import [java.security AccessControlContext Principal]) (:import [javax.security.auth Subject]) (:use [clojure test]) ) (def test-subject (let [rc (ReqContext/context) expected (Subject.)] (is (not (.isReadOnly expected))) (.setSubject rc expected) (is (= (.subject rc) expected)) ; Change the Subject by setting read-only. (.setReadOnly expected) (.setSubject rc expected) (is (= (.subject rc) expected)) ) ) (deftest test-remote-address (let [rc (ReqContext/context) expected (InetAddress/getByAddress (.getBytes "ABCD"))] (.setRemoteAddress rc expected) (is (= (.remoteAddress rc) expected)) ) ) (deftest test-principal-returns-null-when-no-subject (let [rc (ReqContext/context)] (.setSubject rc (Subject.)) (is (nil? (.principal rc))) ) ) (def principal-name "Test Principal") (defn TestPrincipal [] (reify Principal (^String getName [this] principal-name) ) ) (deftest test-principal (let [p (TestPrincipal) principals (hash-set p) creds (hash-set) s (Subject. false principals creds, creds) rc (ReqContext/context)] (.setSubject rc s) (is (not (nil? (.principal rc)))) (is (= (-> rc .principal .getName) principal-name)) ) ) ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/SaslTransportPlugin_test.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.security.auth.SaslTransportPlugin-test (:use [clojure test]) (import [backtype.storm.security.auth SaslTransportPlugin$User]) ) (deftest test-User-name (let [nam "Andy" user (SaslTransportPlugin$User. nam)] (are [a b] (= a b) nam (.toString user) (.getName user) (.toString user) (.hashCode nam) (.hashCode user) ) ) ) (deftest test-User-equals (let [nam "Andy" user1 (SaslTransportPlugin$User. nam) user2 (SaslTransportPlugin$User. nam) user3 (SaslTransportPlugin$User. "Bobby")] (is (-> user1 (.equals user1))) (is (-> user1 (.equals user2))) (is (not (-> user1 (.equals nil)))) (is (not (-> user1 (.equals "Potato")))) (is (not (-> user1 (.equals user3)))) ) ) ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.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.security.auth.ThriftClient-test (:use [backtype.storm config]) (:use [clojure test]) (:import [backtype.storm.security.auth ThriftClient]) (:import [org.apache.thrift7.transport TTransportException]) ) (deftest test-ctor-throws-if-port-invalid (let [conf (read-default-config) timeout (Integer. 30)] (is (thrown? java.lang.IllegalArgumentException (ThriftClient. conf "bogushost" -1 timeout))) (is (thrown? java.lang.IllegalArgumentException (ThriftClient. conf "bogushost" 0 timeout))) ) ) (deftest test-ctor-throws-if-host-not-set (let [conf (read-default-config) timeout (Integer. 60)] (is (thrown? TTransportException (ThriftClient. conf "" 4242 timeout))) (is (thrown? IllegalArgumentException (ThriftClient. conf nil 4242 timeout))) ) ) ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.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.security.auth.ThriftServer-test (:use [backtype.storm config]) (:use [clojure test]) (:import [backtype.storm.security.auth ThriftServer]) (:import [org.apache.thrift7.transport TTransportException]) ) (deftest test-stop-checks-for-null (let [server (ThriftServer. (read-default-config) nil 12345)] (.stop server))) (deftest test-isServing-checks-for-null (let [server (ThriftServer. (read-default-config) nil 12345)] (is (not (.isServing server))))) ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/auth_test.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.security.auth.auth-test (:use [clojure test]) (:require [backtype.storm.daemon [nimbus :as nimbus]]) (:import [org.apache.thrift7 TException]) (:import [org.apache.thrift7.transport TTransportException]) (:import [java.nio ByteBuffer]) (:import [backtype.storm Config]) (:import [backtype.storm.utils NimbusClient]) (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient ReqContext]) (:use [backtype.storm bootstrap util]) (:use [backtype.storm.daemon common]) (:use [backtype.storm bootstrap testing]) (:import [backtype.storm.generated Nimbus Nimbus$Client]) ) (bootstrap) (def nimbus-timeout (Integer. 30)) (defn mk-authorization-handler [storm-conf] (let [klassname (storm-conf NIMBUS-AUTHORIZER) aznClass (if klassname (Class/forName klassname)) aznHandler (if aznClass (.newInstance aznClass))] (if aznHandler (.prepare aznHandler storm-conf)) (log-debug "authorization class name:" klassname " class:" aznClass " handler:" aznHandler) aznHandler )) (defn nimbus-data [storm-conf inimbus] (let [forced-scheduler (.getForcedScheduler inimbus)] {:conf storm-conf :inimbus inimbus :authorization-handler (mk-authorization-handler storm-conf) :submitted-count (atom 0) :storm-cluster-state nil :submit-lock (Object.) :heartbeats-cache (atom {}) :downloaders nil :uploaders nil :uptime (uptime-computer) :validator nil :timer nil :scheduler nil })) (defn check-authorization! [nimbus storm-name storm-conf operation] (let [aclHandler (:authorization-handler nimbus)] (log-debug "check-authorization with handler: " aclHandler) (if aclHandler (if-not (.permit aclHandler (ReqContext/context) operation (if storm-conf storm-conf {TOPOLOGY-NAME storm-name})) (throw (RuntimeException. (str operation " on topology " storm-name " is not authorized"))) )))) (defn dummy-service-handler [conf inimbus] (let [nimbus (nimbus-data conf inimbus)] (reify Nimbus$Iface (^void submitTopologyWithOpts [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology ^SubmitOptions submitOptions] (check-authorization! nimbus storm-name nil "submitTopology")) (^void killTopology [this ^String storm-name] (check-authorization! nimbus storm-name nil "killTopology")) (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options] (check-authorization! nimbus storm-name nil "killTopology")) (^void rebalance [this ^String storm-name ^RebalanceOptions options] (check-authorization! nimbus storm-name nil "rebalance")) (activate [this storm-name] (check-authorization! nimbus storm-name nil "activate")) (deactivate [this storm-name] (check-authorization! nimbus storm-name nil "deactivate")) (beginFileUpload [this]) (^void uploadChunk [this ^String location ^ByteBuffer chunk]) (^void finishFileUpload [this ^String location]) (^String beginFileDownload [this ^String file]) (^ByteBuffer downloadChunk [this ^String id]) (^String getNimbusConf [this]) (^String getTopologyConf [this ^String id]) (^StormTopology getTopology [this ^String id]) (^StormTopology getUserTopology [this ^String id]) (^ClusterSummary getClusterInfo [this]) (^TopologyInfo getTopologyInfo [this ^String storm-id])))) (defn launch-server [server-port login-cfg aznClass transportPluginClass] (let [conf1 (merge (read-storm-config) {NIMBUS-AUTHORIZER aznClass NIMBUS-HOST "localhost" NIMBUS-THRIFT-PORT server-port STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass}) conf (if login-cfg (merge conf1 {"java.security.auth.login.config" login-cfg}) conf1) nimbus (nimbus/standalone-nimbus) service-handler (dummy-service-handler conf nimbus) server (ThriftServer. conf (Nimbus$Processor. service-handler) (int (conf NIMBUS-THRIFT-PORT)))] (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop server)))) (.start (Thread. #(.serve server))) (wait-for-condition #(.isServing server)) server )) (defmacro with-server [args & body] `(let [server# (launch-server ~@args)] ~@body (.stop server#) )) (deftest Simple-authentication-test (let [a-port (available-port)] (with-server [a-port nil nil "backtype.storm.security.auth.SimpleTransportPlugin"] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) nimbus_client (.getClient client)] (.activate nimbus_client "security_auth_test_topology") (.close client)) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"})] (testing "(Negative authentication) Server: Simple vs. Client: Digest" (is (thrown-cause? java.net.SocketTimeoutException (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))))) (deftest positive-authorization-test (let [a-port (available-port)] (with-server [a-port nil "backtype.storm.security.auth.authorizer.NoopAuthorizer" "backtype.storm.security.auth.SimpleTransportPlugin"] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) nimbus_client (.getClient client)] (testing "(Positive authorization) Authorization plugin should accept client request" (.activate nimbus_client "security_auth_test_topology")) (.close client))))) (deftest deny-authorization-test (let [a-port (available-port)] (with-server [a-port nil "backtype.storm.security.auth.authorizer.DenyAuthorizer" "backtype.storm.security.auth.SimpleTransportPlugin"] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin" Config/NIMBUS_HOST "localhost" Config/NIMBUS_THRIFT_PORT a-port Config/NIMBUS_TASK_TIMEOUT_SECS nimbus-timeout}) client (NimbusClient/getConfiguredClient storm-conf) nimbus_client (.getClient client)] (testing "(Negative authorization) Authorization plugin should reject client request" (is (thrown? TTransportException (.activate nimbus_client "security_auth_test_topology")))) (.close client))))) (deftest digest-authentication-test (let [a-port (available-port)] (with-server [a-port "test/clj/backtype/storm/security/auth/jaas_digest.conf" nil "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) nimbus_client (.getClient client)] (testing "(Positive authentication) valid digest authentication" (.activate nimbus_client "security_auth_test_topology")) (.close client)) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) nimbus_client (.getClient client)] (testing "(Negative authentication) Server: Digest vs. Client: Simple" (is (thrown-cause? java.net.SocketTimeoutException (.activate nimbus_client "security_auth_test_topology")))) (.close client)) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_bad_password.conf"})] (testing "(Negative authentication) Invalid password" (is (thrown? TTransportException (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_unknown_user.conf"})] (testing "(Negative authentication) Unknown user" (is (thrown? TTransportException (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/nonexistent.conf"})] (testing "(Negative authentication) nonexistent configuration file" (is (thrown? RuntimeException (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_missing_client.conf"})] (testing "(Negative authentication) Missing client" (is (thrown-cause? java.io.IOException (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))))) (deftest test-GetTransportPlugin-throws-RuntimeException (let [conf (merge (read-storm-config) {Config/STORM_THRIFT_TRANSPORT_PLUGIN "null.invalid"})] (is (thrown? RuntimeException (AuthUtils/GetTransportPlugin conf nil))))) ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/jaas_digest.conf ================================================ /* This sample file illustrates how Digest authentication should be configured */ StormServer { org.apache.zookeeper.server.auth.DigestLoginModule required user_super="adminsecret" user_bob="bobsecret"; }; StormClient { org.apache.zookeeper.server.auth.DigestLoginModule required username="bob" password="bobsecret"; }; ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/jaas_digest_bad_password.conf ================================================ /* This sample file containes incorrect password of a user. We use this file for negative test. */ StormServer { org.apache.zookeeper.server.auth.DigestLoginModule required user_super="adminsecret" user_bob="bobsecret"; }; StormClient { org.apache.zookeeper.server.auth.DigestLoginModule required username="bob" password="bad_password"; }; ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/jaas_digest_missing_client.conf ================================================ StormServer { org.apache.zookeeper.server.auth.DigestLoginModule required user_super="adminsecret" user_bob="bobsecret"; }; ================================================ FILE: storm-core/test/clj/backtype/storm/security/auth/jaas_digest_unknown_user.conf ================================================ /* This sample file containes an unauthorized user. We use this file for negative test. */ StormServer { org.apache.zookeeper.server.auth.DigestLoginModule required user_super="adminsecret" user_bob="bobsecret"; }; StormClient { org.apache.zookeeper.server.auth.DigestLoginModule required username="unknown_user" password="some_password"; }; ================================================ FILE: storm-core/test/clj/backtype/storm/security/serialization/BlowfishTupleSerializer_test.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.security.serialization.BlowfishTupleSerializer-test (:use [clojure test] [backtype.storm.util :only (exception-cause?)] [clojure.string :only (join split)] ) (:import [backtype.storm.security.serialization BlowfishTupleSerializer] [backtype.storm.utils ListDelegate] [com.esotericsoftware.kryo Kryo] [com.esotericsoftware.kryo.io Input Output] ) ) (deftest test-constructor-throws-on-null-key (is (thrown? RuntimeException (new BlowfishTupleSerializer nil {})) "Throws RuntimeException when no encryption key is given.") ) (deftest test-constructor-throws-on-invalid-key ; The encryption key must be hexadecimal. (let [conf {BlowfishTupleSerializer/SECRET_KEY "0123456789abcdefg"}] (is (thrown? RuntimeException (new BlowfishTupleSerializer nil conf)) "Throws RuntimeException when an invalid encryption key is given.") ) ) (deftest test-encrypts-and-decrypts-message (let [ test-text (str "Tetraodontidae is a family of primarily marine and estuarine fish of the order" " Tetraodontiformes. The family includes many familiar species, which are" " variously called pufferfish, puffers, balloonfish, blowfish, bubblefish," " globefish, swellfish, toadfish, toadies, honey toads, sugar toads, and sea" " squab.[1] They are morphologically similar to the closely related" " porcupinefish, which have large external spines (unlike the thinner, hidden" " spines of Tetraodontidae, which are only visible when the fish has puffed up)." " The scientific name refers to the four large teeth, fused into an upper and" " lower plate, which are used for crushing the shells of crustaceans and" " mollusks, their natural prey." ) kryo (new Kryo) arbitrary-key "7dd6fb3203878381b08f9c89d25ed105" storm_conf {BlowfishTupleSerializer/SECRET_KEY arbitrary-key} writer-bts (new BlowfishTupleSerializer kryo storm_conf) reader-bts (new BlowfishTupleSerializer kryo storm_conf) buf-size 1024 output (new Output buf-size buf-size) input (new Input buf-size) strlist (split test-text #" ") delegate (new ListDelegate) ] (-> delegate (.addAll strlist)) (-> writer-bts (.write kryo output delegate)) (.setBuffer input (.getBuffer output)) (is (= test-text (join " " (map (fn [e] (str e)) (-> reader-bts (.read kryo input ListDelegate) (.toArray)))) ) "Reads a string encrypted by another instance with a shared key" ) ) ) ================================================ FILE: storm-core/test/clj/backtype/storm/serialization/SerializationFactory_test.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.serialization.SerializationFactory-test (:import [backtype.storm Config]) (:import [backtype.storm.security.serialization BlowfishTupleSerializer]) (:import [backtype.storm.serialization SerializationFactory]) (:import [backtype.storm.utils ListDelegate]) (:use [backtype.storm config]) (:use [clojure test]) ) (deftest test-registers-default-when-not-in-conf (let [conf (read-default-config) klass-name (get conf Config/TOPOLOGY_TUPLE_SERIALIZER) configured-class (Class/forName klass-name) kryo (SerializationFactory/getKryo conf)] (is (= configured-class (.getClass (.getSerializer kryo ListDelegate)))) ) ) (deftest test-throws-runtimeexception-when-no-such-class (let [conf (merge (read-default-config) {Config/TOPOLOGY_TUPLE_SERIALIZER "null.this.class.does.not.exist"})] (is (thrown? RuntimeException (SerializationFactory/getKryo conf))) ) ) (deftest test-registeres-when-valid-class-name (let [arbitrary-class-name (String. "backtype.storm.security.serialization.BlowfishTupleSerializer") serializer-class (Class/forName arbitrary-class-name) arbitrary-key "0123456789abcdef" conf (merge (read-default-config) {Config/TOPOLOGY_TUPLE_SERIALIZER arbitrary-class-name BlowfishTupleSerializer/SECRET_KEY arbitrary-key}) kryo (SerializationFactory/getKryo conf)] (is (= serializer-class (.getClass (.getSerializer kryo ListDelegate)))) ) ) ================================================ FILE: storm-core/test/clj/backtype/storm/serialization_test.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.serialization-test (:use [clojure test]) (:import [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer KryoValuesSerializer KryoValuesDeserializer]) (:import [backtype.storm.testing TestSerObject TestKryoDecorator]) (:use [backtype.storm util config]) ) (defn mk-conf [extra] (merge (read-default-config) extra)) (defn serialize [vals conf] (let [serializer (KryoValuesSerializer. (mk-conf conf))] (.serialize serializer vals) )) (defn deserialize [bytes conf] (let [deserializer (KryoValuesDeserializer. (mk-conf conf))] (.deserialize deserializer bytes) )) (defn roundtrip ([vals] (roundtrip vals {})) ([vals conf] (deserialize (serialize vals conf) conf))) (deftest test-java-serialization (letlocals (bind obj (TestSerObject. 1 2)) (is (thrown? Exception (roundtrip [obj] {TOPOLOGY-KRYO-REGISTER {"backtype.storm.testing.TestSerObject" nil} TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))) (is (= [obj] (roundtrip [obj] {TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION true}))))) (deftest test-kryo-decorator (letlocals (bind obj (TestSerObject. 1 2)) (is (thrown? Exception (roundtrip [obj] {TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))) (is (= [obj] (roundtrip [obj] {TOPOLOGY-KRYO-DECORATORS ["backtype.storm.testing.TestKryoDecorator"] TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))))) (defn mk-string [size] (let [builder (StringBuilder.)] (doseq [i (range size)] (.append builder "a")) (.toString builder))) (defn is-roundtrip [vals] (is (= vals (roundtrip vals)))) (deftest test-string-serialization (is-roundtrip ["a" "bb" "cde"]) (is-roundtrip [(mk-string (* 64 1024))]) (is-roundtrip [(mk-string (* 1024 1024))]) (is-roundtrip [(mk-string (* 1024 1024 2))]) ) (deftest test-clojure-serialization (is-roundtrip [:a]) (is-roundtrip [["a" 1 2 :a] 2 "aaa"]) (is-roundtrip [#{:a :b :c}]) (is-roundtrip [#{:a :b} 1 2 ["a" 3 5 #{5 6}]]) (is-roundtrip [{:a [1 2 #{:a :b 1}] :b 3}])) ================================================ FILE: storm-core/test/clj/backtype/storm/subtopology_test.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.subtopology-test (:use [clojure test]) (:import [backtype.storm.topology TopologyBuilder]) (:import [backtype.storm.testing TestWordSpout PrepareBatchBolt BatchRepeatA BatchProcessWord BatchNumberList]) (:import [backtype.storm.coordination BatchSubtopologyBuilder]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) ;; todo: need to configure coordinatedbolt with streams that aren't subscribed to, should auto-anchor those to the final ;; coordination tuple... find all streams that aren't subscribed to ;; having trouble with this test, commenting for now ;; (deftest test-batch-subtopology ;; (with-local-cluster [cluster :supervisors 4] ;; (letlocals ;; (bind builder (TopologyBuilder.)) ;; (.setSpout builder "spout" (TestWordSpout.)) ;; (-> (.setBolt builder "identity" (PrepareBatchBolt. (Fields. ["id" "word"])) 3) ;; (.shuffleGrouping "spout") ;; ) ;; (bind batch-builder (BatchSubtopologyBuilder. "for-a" (BatchRepeatA.) 2)) ;; (-> (.getMasterDeclarer batch-builder) ;; (.shuffleGrouping "identity")) ;; (-> (.setBolt batch-builder "process" (BatchProcessWord.) 2) ;; (.fieldsGrouping "for-a" "multi" (Fields. ["id"]))) ;; (-> (.setBolt batch-builder "joiner" (BatchNumberList. "for-a") 2) ;; (.fieldsGrouping "process" (Fields. ["id"])) ;; (.fieldsGrouping "for-a" "single" (Fields. ["id"])) ;; ) ;; ;; (.extendTopology batch-builder builder) ;; ;; (bind results (complete-topology cluster ;; (.createTopology builder) ;; :storm-conf {TOPOLOGY-DEBUG true} ;; :mock-sources {"spout" [ ;; ["ccacccaa"] ;; ["bbb"] ;; ["ba"] ;; ]} ;; )) ;; (is (ms= [ ;; ["ccacccaa" [2 6 7]] ;; ["bbb" []] ;; ["ba" [1]] ;; ] ;; (read-tuples results "joiner"))) ;; ))) ================================================ FILE: storm-core/test/clj/backtype/storm/supervisor_test.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.supervisor-test (:use [clojure test]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) (:require [backtype.storm.daemon [worker :as worker] [supervisor :as supervisor]]) ) (bootstrap) (defn worker-assignment "Return [storm-id executors]" [cluster supervisor-id port] (let [state (:storm-cluster-state cluster) slot-assigns (for [storm-id (.assignments state nil)] (let [executors (-> (.assignment-info state storm-id nil) :executor->node+port reverse-map (get [supervisor-id port] ))] (when executors [storm-id executors]) )) ret (find-first not-nil? slot-assigns)] (when-not ret (throw-runtime "Could not find assignment for worker")) ret )) (defn heartbeat-worker [supervisor port storm-id executors] (let [conf (.get-conf supervisor)] (worker/do-heartbeat {:conf conf :port port :storm-id storm-id :executors executors :worker-id (find-worker-id conf port)}))) (defn heartbeat-workers [cluster supervisor-id ports] (let [sup (get-supervisor cluster supervisor-id)] (doseq [p ports] (let [[storm-id executors] (worker-assignment cluster supervisor-id p)] (heartbeat-worker sup p storm-id executors) )))) (defn validate-launched-once [launched supervisor->ports storm-id] (let [counts (map count (vals launched)) launched-supervisor->ports (apply merge-with concat (for [[s p] (keys launched)] {s [p]} ))] (is (every? (partial = 1) counts)) (is (= launched-supervisor->ports supervisor->ports)) )) (deftest launches-assignment (with-simulated-time-local-cluster [cluster :supervisors 0 :daemon-conf {NIMBUS-REASSIGN false SUPERVISOR-WORKER-START-TIMEOUT-SECS 5 SUPERVISOR-WORKER-TIMEOUT-SECS 15 SUPERVISOR-MONITOR-FREQUENCY-SECS 3}] (letlocals (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 4)} {})) (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4])) (bind changed (capture-changed-workers (submit-mocked-assignment (:nimbus cluster) "test" {TOPOLOGY-WORKERS 3} topology {1 "1" 2 "1" 3 "1" 4 "1"} {[1] ["sup1" 1] [2] ["sup1" 2] [3] ["sup1" 3] [4] ["sup1" 3] }) (advance-cluster-time cluster 2) (heartbeat-workers cluster "sup1" [1 2 3]) (advance-cluster-time cluster 10))) (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test")) (is (empty? (:shutdown changed))) (validate-launched-once (:launched changed) {"sup1" [1 2 3]} storm-id) (bind changed (capture-changed-workers (doseq [i (range 10)] (heartbeat-workers cluster "sup1" [1 2 3]) (advance-cluster-time cluster 10)) )) (is (empty? (:shutdown changed))) (is (empty? (:launched changed))) (bind changed (capture-changed-workers (heartbeat-workers cluster "sup1" [1 2]) (advance-cluster-time cluster 10) )) (validate-launched-once (:launched changed) {"sup1" [3]} storm-id) (is (= {["sup1" 3] 1} (:shutdown changed))) ))) (deftest test-multiple-active-storms-multiple-supervisors (with-simulated-time-local-cluster [cluster :supervisors 0 :daemon-conf {NIMBUS-REASSIGN false SUPERVISOR-WORKER-START-TIMEOUT-SECS 5 SUPERVISOR-WORKER-TIMEOUT-SECS 15 SUPERVISOR-MONITOR-FREQUENCY-SECS 3}] (letlocals (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 4)} {})) (bind topology2 (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} {})) (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4])) (bind sup2 (add-supervisor cluster :id "sup2" :ports [1 2])) (bind changed (capture-changed-workers (submit-mocked-assignment (:nimbus cluster) "test" {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 40} topology {1 "1" 2 "1" 3 "1" 4 "1"} {[1] ["sup1" 1] [2] ["sup1" 2] [3] ["sup2" 1] [4] ["sup2" 1] }) (advance-cluster-time cluster 2) (heartbeat-workers cluster "sup1" [1 2]) (heartbeat-workers cluster "sup2" [1]) )) (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test")) (is (empty? (:shutdown changed))) (validate-launched-once (:launched changed) {"sup1" [1 2] "sup2" [1]} storm-id) (bind changed (capture-changed-workers (submit-mocked-assignment (:nimbus cluster) "test2" {TOPOLOGY-WORKERS 2} topology2 {1 "1" 2 "1" 3 "1"} {[1] ["sup1" 3] [2] ["sup1" 3] [3] ["sup2" 2] }) (advance-cluster-time cluster 2) (heartbeat-workers cluster "sup1" [3]) (heartbeat-workers cluster "sup2" [2]) )) (bind storm-id2 (get-storm-id (:storm-cluster-state cluster) "test2")) (is (empty? (:shutdown changed))) (validate-launched-once (:launched changed) {"sup1" [3] "sup2" [2]} storm-id2) (bind changed (capture-changed-workers (.killTopology (:nimbus cluster) "test") (doseq [i (range 4)] (advance-cluster-time cluster 8) (heartbeat-workers cluster "sup1" [1 2 3]) (heartbeat-workers cluster "sup2" [1 2]) ))) (is (empty? (:shutdown changed))) (is (empty? (:launched changed))) (bind changed (capture-changed-workers (advance-cluster-time cluster 12) )) (is (empty? (:launched changed))) (is (= {["sup1" 1] 1 ["sup1" 2] 1 ["sup2" 1] 1} (:shutdown changed))) (bind changed (capture-changed-workers (doseq [i (range 10)] (heartbeat-workers cluster "sup1" [3]) (heartbeat-workers cluster "sup2" [2]) (advance-cluster-time cluster 10) ))) (is (empty? (:shutdown changed))) (is (empty? (:launched changed))) ;; TODO check that downloaded code is cleaned up only for the one storm ))) (defn get-heartbeat [cluster supervisor-id] (.supervisor-info (:storm-cluster-state cluster) supervisor-id)) (defn check-heartbeat [cluster supervisor-id within-secs] (let [hb (get-heartbeat cluster supervisor-id) time-secs (:time-secs hb) now (current-time-secs) delta (- now time-secs)] (is (>= delta 0)) (is (<= delta within-secs)) )) (deftest heartbeats-to-nimbus (with-simulated-time-local-cluster [cluster :supervisors 0 :daemon-conf {SUPERVISOR-WORKER-START-TIMEOUT-SECS 15 SUPERVISOR-HEARTBEAT-FREQUENCY-SECS 3}] (letlocals (bind sup1 (add-supervisor cluster :id "sup" :ports [5 6 7])) (advance-cluster-time cluster 4) (bind hb (get-heartbeat cluster "sup")) (is (= #{5 6 7} (set (:meta hb)))) (check-heartbeat cluster "sup" 3) (advance-cluster-time cluster 3) (check-heartbeat cluster "sup" 3) (advance-cluster-time cluster 3) (check-heartbeat cluster "sup" 3) (advance-cluster-time cluster 15) (check-heartbeat cluster "sup" 3) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 4)} {})) ;; prevent them from launching by capturing them (capture-changed-workers (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 2} topology)) (advance-cluster-time cluster 3) (check-heartbeat cluster "sup" 3) (advance-cluster-time cluster 3) (check-heartbeat cluster "sup" 3) (advance-cluster-time cluster 3) (check-heartbeat cluster "sup" 3) (advance-cluster-time cluster 20) (check-heartbeat cluster "sup" 3) ))) (deftest test-workers-go-bananas ;; test that multiple workers are started for a port, and test that ;; supervisor shuts down propertly (doesn't shutdown the most ;; recently launched one, checks heartbeats correctly, etc.) ) (deftest downloads-code ) (deftest test-stateless ) (deftest cleans-up-on-unassign ;; TODO just do reassign, and check that cleans up worker states after killing but doesn't get rid of downloaded code ) ================================================ FILE: storm-core/test/clj/backtype/storm/testing4j_test.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.testing4j-test (:use [clojure.test]) (:use [backtype.storm config clojure testing util]) (:require [backtype.storm.integration-test :as it]) (:require [backtype.storm.thrift :as thrift]) (:import [backtype.storm Testing Config ILocalCluster]) (:import [backtype.storm.tuple Values Tuple]) (:import [backtype.storm.utils Time Utils]) (:import [backtype.storm.testing MkClusterParam TestJob MockedSources TestWordSpout TestWordCounter TestGlobalCount TestAggregatesCounter CompleteTopologyParam AckFailMapTracker MkTupleParam])) (deftest test-with-simulated-time (is (= false (Time/isSimulating))) (Testing/withSimulatedTime (fn [] (is (= true (Time/isSimulating))))) (is (= false (Time/isSimulating)))) (deftest test-with-local-cluster (let [mk-cluster-param (doto (MkClusterParam.) (.setSupervisors (int 2)) (.setPortsPerSupervisor (int 5))) daemon-conf (doto (Config.) (.put SUPERVISOR-ENABLE false) (.put TOPOLOGY-ACKER-EXECUTORS 0))] (Testing/withLocalCluster mk-cluster-param (reify TestJob (^void run [this ^ILocalCluster cluster] (is (not (nil? cluster))) (is (not (nil? (.getState cluster)))) (is (not (nil? (:nimbus (.getState cluster)))))))))) (deftest test-with-simulated-time-local-cluster (let [mk-cluster-param (doto (MkClusterParam.) (.setSupervisors (int 2))) daemon-conf (doto (Config.) (.put SUPERVISOR-ENABLE false) (.put TOPOLOGY-ACKER-EXECUTORS 0))] (is (not (Time/isSimulating))) (Testing/withSimulatedTimeLocalCluster mk-cluster-param (reify TestJob (^void run [this ^ILocalCluster cluster] (is (not (nil? cluster))) (is (not (nil? (.getState cluster)))) (is (not (nil? (:nimbus (.getState cluster))))) (is (Time/isSimulating))))) (is (not (Time/isSimulating))))) (deftest test-complete-topology (doseq [zmq-on? [true false] :let [daemon-conf (doto (Config.) (.put STORM-LOCAL-MODE-ZMQ zmq-on?)) mk-cluster-param (doto (MkClusterParam.) (.setSupervisors (int 4)) (.setDaemonConf daemon-conf))]] (Testing/withSimulatedTimeLocalCluster (reify TestJob (^void run [this ^ILocalCluster cluster] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :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.)) }) mocked-sources (doto (MockedSources.) (.addMockData "1" (into-array Values [(Values. (into-array ["nathan"])) (Values. (into-array ["bob"])) (Values. (into-array ["joey"])) (Values. (into-array ["nathan"]))]) )) storm-conf (doto (Config.) (.setNumWorkers 2)) complete-topology-param (doto (CompleteTopologyParam.) (.setMockedSources mocked-sources) (.setStormConf storm-conf)) results (Testing/completeTopology cluster topology complete-topology-param)] (is (Testing/multiseteq [["nathan"] ["bob"] ["joey"] ["nathan"]] (Testing/readTuples results "1"))) (is (Testing/multiseteq [["nathan" 1] ["nathan" 2] ["bob" 1] ["joey" 1]] (read-tuples results "2"))) (is (= [[1] [2] [3] [4]] (Testing/readTuples results "3"))) (is (= [[1] [2] [3] [4]] (Testing/readTuples results "4"))) )))))) (deftest test-with-tracked-cluster (Testing/withTrackedCluster (reify TestJob (^void run [this ^ILocalCluster cluster] (let [[feeder checker] (it/ack-tracking-feeder ["num"]) tracked (Testing/mkTrackedTopology cluster (topology {"1" (spout-spec feeder)} {"2" (bolt-spec {"1" :shuffle} it/identity-bolt) "3" (bolt-spec {"1" :shuffle} it/identity-bolt) "4" (bolt-spec {"2" :shuffle "3" :shuffle} (it/agg-bolt 4))}))] (.submitTopology cluster "test-acking2" (Config.) (.getTopology tracked)) (.feed feeder [1]) (Testing/trackedWait tracked (int 1)) (checker 0) (.feed feeder [1]) (Testing/trackedWait tracked (int 1)) (checker 2) ))))) (deftest test-advance-cluster-time (let [daemon-conf (doto (Config.) (.put TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true)) mk-cluster-param (doto (MkClusterParam.) (.setDaemonConf daemon-conf))] (Testing/withSimulatedTimeLocalCluster mk-cluster-param (reify TestJob (^void run [this ^ILocalCluster cluster] (let [feeder (feeder-spout ["field1"]) tracker (AckFailMapTracker.) _ (.setAckFailDelegate feeder tracker) topology (thrift/mk-topology {"1" (thrift/mk-spout-spec feeder)} {"2" (thrift/mk-bolt-spec {"1" :global} it/ack-every-other)}) storm-conf (doto (Config.) (.put TOPOLOGY-MESSAGE-TIMEOUT-SECS 10))] (.submitTopology cluster "timeout-tester" storm-conf topology) (.feed feeder ["a"] 1) (.feed feeder ["b"] 2) (.feed feeder ["c"] 3) (Testing/advanceClusterTime cluster (int 9)) (it/assert-acked tracker 1 3) (is (not (.isFailed tracker 2))) (Testing/advanceClusterTime cluster (int 12)) (it/assert-failed tracker 2) )))))) (deftest test-test-tuple (letlocals ;; test the one-param signature (bind ^Tuple tuple (Testing/testTuple ["james" "bond"])) (is (= ["james" "bond"] (.getValues tuple))) (is (= Utils/DEFAULT_STREAM_ID (.getSourceStreamId tuple))) (is (= ["field1" "field2"] (-> tuple .getFields .toList))) (is (= "component" (.getSourceComponent tuple))) ;; test the two-params signature (bind mk-tuple-param (MkTupleParam.)) (doto mk-tuple-param (.setStream "test-stream") (.setComponent "test-component") (.setFields (into-array String ["fname" "lname"]))) (bind ^Tuple tuple (Testing/testTuple ["james" "bond"] mk-tuple-param)) (is (= ["james" "bond"] (.getValues tuple))) (is (= "test-stream" (.getSourceStreamId tuple))) (is (= ["fname" "lname"] (-> tuple .getFields .toList))) (is (= "test-component" (.getSourceComponent tuple))))) ================================================ FILE: storm-core/test/clj/backtype/storm/tick_tuple_test.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.tick-tuple-test (:use [clojure test]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common])) (bootstrap) (defbolt noop-bolt ["tuple"] {:prepare true} [conf context collector] (bolt (execute [tuple]))) (defspout noop-spout ["tuple"] [conf context collector] (spout (nextTuple []))) (deftest test-tick-tuple-works-with-system-bolt (with-simulated-time-local-cluster [cluster] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec noop-spout)} {"2" (thrift/mk-bolt-spec {"1" ["tuple"]} noop-bolt)})] (try (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-TICK-TUPLE-FREQ-SECS 1} topology) (advance-cluster-time cluster 2) ;; if reaches here, it means everything works ok. (is true) (catch Exception e (is false)))))) ================================================ FILE: storm-core/test/clj/backtype/storm/transactional_test.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.transactional-test (:use [clojure test]) (:import [backtype.storm.topology TopologyBuilder]) (:import [backtype.storm.transactional TransactionalSpoutCoordinator ITransactionalSpout ITransactionalSpout$Coordinator TransactionAttempt TransactionalTopologyBuilder]) (:import [backtype.storm.transactional.state TransactionalState RotatingTransactionalState RotatingTransactionalState$StateInitializer]) (:import [backtype.storm.testing CountingBatchBolt MemoryTransactionalSpout KeyedCountingBatchBolt KeyedCountingCommitterBolt KeyedSummingBatchBolt IdentityBolt CountingCommitBolt OpaqueMemoryTransactionalSpout]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) (bootstrap) ;; Testing TODO: ;; * Test that it repeats the meta for a partitioned state (test partitioned emitter on its own) ;; * Test that partitioned state emits nothing for the partition if it has seen a future transaction for that partition (test partitioned emitter on its own) (defn mk-coordinator-state-changer [atom] (TransactionalSpoutCoordinator. (reify ITransactionalSpout (getComponentConfiguration [this] nil) (getCoordinator [this conf context] (reify ITransactionalSpout$Coordinator (isReady [this] (not (nil? @atom))) (initializeTransaction [this txid prevMetadata] @atom ) (close [this] ))) ))) (def BATCH-STREAM TransactionalSpoutCoordinator/TRANSACTION_BATCH_STREAM_ID) (def COMMIT-STREAM TransactionalSpoutCoordinator/TRANSACTION_COMMIT_STREAM_ID) (defn mk-spout-capture [capturer] (SpoutOutputCollector. (reify ISpoutOutputCollector (emit [this stream-id tuple message-id] (swap! capturer update-in [stream-id] (fn [oldval] (concat oldval [{:tuple tuple :id message-id}]))) [] )))) (defn normalize-tx-tuple [values] (-> values vec (update 0 #(-> % .getTransactionId .intValue)))) (defn verify-and-reset! [expected-map emitted-map-atom] (let [results @emitted-map-atom] (dorun (map-val (fn [tuples] (doseq [t tuples] (is (= (-> t :tuple first) (:id t))) )) results)) (is (= expected-map (map-val (fn [tuples] (map (comp normalize-tx-tuple #(take 2 %) :tuple) tuples)) results ))) (reset! emitted-map-atom {}) )) (defn get-attempts [capture-atom stream] (map :id (get @capture-atom stream))) (defn get-commit [capture-atom] (-> @capture-atom (get COMMIT-STREAM) first :id)) (deftest test-coordinator (let [coordinator-state (atom nil) emit-capture (atom nil)] (with-inprocess-zookeeper zk-port (letlocals (bind coordinator (mk-coordinator-state-changer coordinator-state)) (.open coordinator (merge (read-default-config) {TOPOLOGY-MAX-SPOUT-PENDING 4 TOPOLOGY-TRANSACTIONAL-ID "abc" STORM-ZOOKEEPER-PORT zk-port STORM-ZOOKEEPER-SERVERS ["localhost"] }) nil (mk-spout-capture emit-capture)) (reset! coordinator-state 10) (.nextTuple coordinator) (bind attempts (get-attempts emit-capture BATCH-STREAM)) (bind first-attempt (first attempts)) (verify-and-reset! {BATCH-STREAM [[1 10] [2 10] [3 10] [4 10]]} emit-capture) (.nextTuple coordinator) (verify-and-reset! {} emit-capture) (.fail coordinator (second attempts)) (bind attempts (get-attempts emit-capture BATCH-STREAM)) (bind new-second-attempt (first attempts)) (verify-and-reset! {BATCH-STREAM [[2 10] [3 10] [4 10]]} emit-capture) (is (not= new-second-attempt (second attempts))) (.ack coordinator new-second-attempt) (verify-and-reset! {} emit-capture) (.ack coordinator first-attempt) (bind commit-id (get-commit emit-capture)) (verify-and-reset! {COMMIT-STREAM [[1]]} emit-capture) (reset! coordinator-state 12) (.ack coordinator commit-id) (bind commit-id (get-commit emit-capture)) (verify-and-reset! {COMMIT-STREAM [[2]] BATCH-STREAM [[5 12]]} emit-capture) (reset! coordinator-state nil) (.ack coordinator commit-id) (verify-and-reset! {} emit-capture) (.fail coordinator (nth attempts 1)) (bind attempts (get-attempts emit-capture BATCH-STREAM)) (verify-and-reset! {BATCH-STREAM [[3 10] [4 10] [5 12]]} emit-capture) (reset! coordinator-state 12) (.nextTuple coordinator) (verify-and-reset! {BATCH-STREAM [[6 12]]} emit-capture) (.ack coordinator (first attempts)) (bind commit-id (get-commit emit-capture)) (verify-and-reset! {COMMIT-STREAM [[3]]} emit-capture) (.ack coordinator (nth attempts 1)) (verify-and-reset! {} emit-capture) (.fail coordinator commit-id) (bind attempts (get-attempts emit-capture BATCH-STREAM)) (verify-and-reset! {BATCH-STREAM [[3 10] [4 10] [5 12] [6 12]]} emit-capture) (.ack coordinator (first attempts)) (bind commit-id (get-commit emit-capture)) (verify-and-reset! {COMMIT-STREAM [[3]]} emit-capture) (.ack coordinator (second attempts)) (.nextTuple coordinator) (verify-and-reset! {} emit-capture) (.ack coordinator commit-id) (verify-and-reset! {COMMIT-STREAM [[4]] BATCH-STREAM [[7 12]]} emit-capture) (.close coordinator) )))) (defn verify-bolt-and-reset! [expected-map emitted-atom] (is (= expected-map @emitted-atom)) (reset! emitted-atom {})) (defn mk-bolt-capture [capturer] (let [adder (fn [amap key newvalue] (update-in amap [key] (fn [ov] (concat ov [newvalue]) )))] (OutputCollector. (reify IOutputCollector (emit [this stream-id anchors values] (swap! capturer adder stream-id values) [] ) (ack [this tuple] (swap! capturer adder :ack (.getValues tuple)) ) (fail [this tuple] (swap! capturer adder :fail (.getValues tuple))) )))) (defn mk-attempt [txid attempt-id] (TransactionAttempt. (BigInteger. (str txid)) attempt-id)) (defn finish! [bolt id] (.finishedId bolt id)) (deftest test-batch-bolt (let [bolt (BatchBoltExecutor. (CountingBatchBolt.)) capture-atom (atom {}) attempt1-1 (mk-attempt 1 1) attempt1-2 (mk-attempt 1 2) attempt2-1 (mk-attempt 2 1) attempt3 (mk-attempt 3 1) attempt4 (mk-attempt 4 1) attempt5 (mk-attempt 5 1) attempt6 (mk-attempt 6 1)] (.prepare bolt {} nil (mk-bolt-capture capture-atom)) ;; test that transactions are independent (.execute bolt (test-tuple [attempt1-1])) (.execute bolt (test-tuple [attempt1-1])) (.execute bolt (test-tuple [attempt1-2])) (.execute bolt (test-tuple [attempt2-1])) (.execute bolt (test-tuple [attempt1-1])) (finish! bolt attempt1-1) (verify-bolt-and-reset! {:ack [[attempt1-1] [attempt1-1] [attempt1-2] [attempt2-1] [attempt1-1]] "default" [[attempt1-1 3]]} capture-atom) (.execute bolt (test-tuple [attempt1-2])) (finish! bolt attempt2-1) (verify-bolt-and-reset! {:ack [[attempt1-2]] "default" [[attempt2-1 1]]} capture-atom) (finish! bolt attempt1-2) (verify-bolt-and-reset! {"default" [[attempt1-2 2]]} capture-atom) )) (defn mk-state-initializer [atom] (reify RotatingTransactionalState$StateInitializer (init [this txid last-state] @atom ))) (defn- to-txid [txid] (BigInteger. (str txid))) (defn- get-state [state txid initializer] (.getState state (to-txid txid) initializer)) (defn- get-state-or-create [state txid initializer] (.getStateOrCreate state (to-txid txid) initializer)) (defn- cleanup-before [state txid] (.cleanupBefore state (to-txid txid))) (deftest test-rotating-transactional-state ;; test strict ordered vs not strict ordered (with-inprocess-zookeeper zk-port (let [conf (merge (read-default-config) {STORM-ZOOKEEPER-PORT zk-port STORM-ZOOKEEPER-SERVERS ["localhost"] }) state (TransactionalState/newUserState conf "id1" {}) strict-rotating (RotatingTransactionalState. state "strict" true) unstrict-rotating (RotatingTransactionalState. state "unstrict" false) init (atom 10) initializer (mk-state-initializer init)] (is (= 10 (get-state strict-rotating 1 initializer))) (is (= 10 (get-state strict-rotating 2 initializer))) (reset! init 20) (is (= 20 (get-state strict-rotating 3 initializer))) (is (= 10 (get-state strict-rotating 1 initializer))) (is (thrown? Exception (get-state strict-rotating 5 initializer))) (is (= 20 (get-state strict-rotating 4 initializer))) (is (= 4 (count (.list state "strict")))) (cleanup-before strict-rotating 3) (is (= 2 (count (.list state "strict")))) (is (nil? (get-state-or-create strict-rotating 5 initializer))) (is (= 20 (get-state-or-create strict-rotating 5 initializer))) (is (nil? (get-state-or-create strict-rotating 6 initializer))) (cleanup-before strict-rotating 6) (is (= 1 (count (.list state "strict")))) (is (= 20 (get-state unstrict-rotating 10 initializer))) (is (= 20 (get-state unstrict-rotating 20 initializer))) (is (nil? (get-state unstrict-rotating 12 initializer))) (is (nil? (get-state unstrict-rotating 19 initializer))) (is (nil? (get-state unstrict-rotating 12 initializer))) (is (= 20 (get-state unstrict-rotating 21 initializer))) (.close state) ))) (defn mk-transactional-source [] (HashMap.)) (defn add-transactional-data [source partition-map] (doseq [[p data] partition-map :let [p (int p)]] (if-not (contains? source p) (.put source p (Collections/synchronizedList (ArrayList.)))) (-> source (.get p) (.addAll data)) )) (defn tracked-captured-topology [cluster topology] (let [{captured :capturer topology :topology} (capture-topology topology) tracked (mk-tracked-topology cluster topology)] (assoc tracked :capturer captured) )) ;; puts its collector and tuples into the global state to be used externally (defbolt controller-bolt {} {:prepare true :params [state-id]} [conf context collector] (let [{tuples :tuples collector-atom :collector} (RegisteredGlobalState/getState state-id)] (reset! collector-atom collector) (reset! tuples []) (bolt (execute [tuple] (swap! tuples conj tuple)) ))) (defmacro with-controller-bolt [[bolt collector-atom tuples-atom] & body] `(let [~collector-atom (atom nil) ~tuples-atom (atom []) id# (RegisteredGlobalState/registerState {:collector ~collector-atom :tuples ~tuples-atom}) ~bolt (controller-bolt id#)] ~@body (RegisteredGlobalState/clearState id#) )) (deftest test-transactional-topology (with-tracked-cluster [cluster] (with-controller-bolt [controller collector tuples] (letlocals (bind data (mk-transactional-source)) (bind builder (TransactionalTopologyBuilder. "id" "spout" (MemoryTransactionalSpout. data (Fields. ["word" "amt"]) 2) 2)) (-> builder (.setBolt "id1" (IdentityBolt. (Fields. ["tx" "word" "amt"])) 3) (.shuffleGrouping "spout")) (-> builder (.setBolt "id2" (IdentityBolt. (Fields. ["tx" "word" "amt"])) 3) (.shuffleGrouping "spout")) (-> builder (.setBolt "global" (CountingBatchBolt.) 1) (.globalGrouping "spout")) (-> builder (.setBolt "gcommit" (CountingCommitBolt.) 1) (.globalGrouping "spout")) (-> builder (.setBolt "sum" (KeyedSummingBatchBolt.) 2) (.fieldsGrouping "id1" (Fields. ["word"]))) (-> builder (.setCommitterBolt "count" (KeyedCountingBatchBolt.) 2) (.fieldsGrouping "id2" (Fields. ["word"]))) (-> builder (.setBolt "count2" (KeyedCountingCommitterBolt.) 3) (.fieldsGrouping "sum" (Fields. ["key"])) (.fieldsGrouping "count" (Fields. ["key"]))) (bind builder (.buildTopologyBuilder builder)) (-> builder (.setBolt "controller" controller 1) (.directGrouping "count2" Constants/COORDINATED_STREAM_ID) (.directGrouping "sum" Constants/COORDINATED_STREAM_ID)) (add-transactional-data data {0 [["dog" 3] ["cat" 4] ["apple" 1] ["dog" 3]] 1 [["cat" 1] ["mango" 4]] 2 [["happy" 11] ["mango" 2] ["zebra" 1]]}) (bind topo-info (tracked-captured-topology cluster (.createTopology builder))) (submit-local-topology (:nimbus cluster) "transactional-test" {TOPOLOGY-MAX-SPOUT-PENDING 2} (:topology topo-info)) (bind ack-tx! (fn [txid] (let [[to-ack not-to-ack] (separate #(-> % (.getValue 0) .getTransactionId (= txid)) @tuples)] (reset! tuples not-to-ack) (doseq [t to-ack] (ack! @collector t))))) (bind fail-tx! (fn [txid] (let [[to-fail not-to-fail] (separate #(-> % (.getValue 0) .getTransactionId (= txid)) @tuples)] (reset! tuples not-to-fail) (doseq [t to-fail] (fail! @collector t))))) ;; only check default streams (bind verify! (fn [expected] (let [results (-> topo-info :capturer .getResults)] (doseq [[component tuples] expected :let [emitted (->> (read-tuples results component "default") (map normalize-tx-tuple))]] (is (ms= tuples emitted))) (.clear results) ))) (tracked-wait topo-info 2) (verify! {"sum" [[1 "dog" 3] [1 "cat" 5] [1 "mango" 6] [1 "happy" 11] [2 "apple" 1] [2 "dog" 3] [2 "zebra" 1]] "count" [] "count2" [] "global" [[1 6] [2 3]] "gcommit" []}) (ack-tx! 1) (tracked-wait topo-info 1) (verify! {"sum" [] "count" [[1 "dog" 1] [1 "cat" 2] [1 "mango" 2] [1 "happy" 1]] "count2" [[1 "dog" 2] [1 "cat" 2] [1 "mango" 2] [1 "happy" 2]] "global" [] "gcommit" [[1 6]]}) (add-transactional-data data {0 [["a" 1] ["b" 2] ["c" 3]] 1 [["d" 4] ["c" 1]] 2 [["a" 2] ["e" 7] ["c" 11]] 3 [["a" 2]]}) (ack-tx! 1) (tracked-wait topo-info 1) (verify! {"sum" [[3 "a" 5] [3 "b" 2] [3 "d" 4] [3 "c" 1] [3 "e" 7]] "count" [] "count2" [] "global" [[3 7]] "gcommit" []}) (ack-tx! 3) (ack-tx! 2) (tracked-wait topo-info 1) (verify! {"sum" [] "count" [[2 "apple" 1] [2 "dog" 1] [2 "zebra" 1]] "count2" [[2 "apple" 2] [2 "dog" 2] [2 "zebra" 2]] "global" [] "gcommit" [[2 3]]}) (fail-tx! 2) (tracked-wait topo-info 2) (verify! {"sum" [[2 "apple" 1] [2 "dog" 3] [2 "zebra" 1] [3 "a" 5] [3 "b" 2] [3 "d" 4] [3 "c" 1] [3 "e" 7]] "count" [] "count2" [] "global" [[2 3] [3 7]] "gcommit" []}) (ack-tx! 2) (tracked-wait topo-info 1) (verify! {"sum" [] "count" [[2 "apple" 1] [2 "dog" 1] [2 "zebra" 1]] "count2" [[2 "apple" 2] [2 "dog" 2] [2 "zebra" 2]] "global" [] "gcommit" [[2 3]]}) (ack-tx! 2) (ack-tx! 3) (tracked-wait topo-info 2) (verify! {"sum" [[4 "c" 14]] "count" [[3 "a" 3] [3 "b" 1] [3 "d" 1] [3 "c" 1] [3 "e" 1]] "count2" [[3 "a" 2] [3 "b" 2] [3 "d" 2] [3 "c" 2] [3 "e" 2]] "global" [[4 2]] "gcommit" [[3 7]]}) (ack-tx! 4) (ack-tx! 3) (tracked-wait topo-info 2) (verify! {"sum" [] "count" [[4 "c" 2]] "count2" [[4 "c" 2]] "global" [[5 0]] "gcommit" [[4 2]]}) (ack-tx! 5) (ack-tx! 4) (tracked-wait topo-info 2) (verify! {"sum" [] "count" [] "count2" [] "global" [[6 0]] "gcommit" [[5 0]]}) (-> topo-info :capturer .getAndClearResults) )))) (deftest test-transactional-topology-restart (with-simulated-time-local-cluster [cluster] (letlocals (bind data (mk-transactional-source)) (bind builder (TransactionalTopologyBuilder. "id" "spout" (MemoryTransactionalSpout. data (Fields. ["word"]) 3) 2)) (-> builder (.setBolt "count" (CountingCommitBolt.) 2) (.globalGrouping "spout")) (add-transactional-data data {0 [["a"] ["b"] ["c"] ["d"]] 1 [["d"] ["c"]] }) (bind results (complete-topology cluster (.buildTopology builder) :cleanup-state false)) (is (ms= [[5] [0] [1] [0]] (->> (read-tuples results "count") (take 4) (map (partial drop 1)) ))) (add-transactional-data data {0 [["a"] ["b"]] }) (bind results (complete-topology cluster (.buildTopology builder))) ;; need to do it this way (check for nothing transaction) because there is one transaction already saved up before that emits nothing (because of how memorytransctionalspout detects partition completion) (is (ms= [[0] [0] [2] [0]] (->> (read-tuples results "count") (take 4) (map (partial drop 1)) ))) ))) (deftest test-opaque-transactional-topology (with-tracked-cluster [cluster] (with-controller-bolt [controller collector tuples] (letlocals (bind data (mk-transactional-source)) (bind builder (TransactionalTopologyBuilder. "id" "spout" (OpaqueMemoryTransactionalSpout. data (Fields. ["word"]) 2) 2)) (-> builder (.setCommitterBolt "count" (KeyedCountingBatchBolt.) 2) (.fieldsGrouping "spout" (Fields. ["word"]))) (bind builder (.buildTopologyBuilder builder)) (-> builder (.setBolt "controller" controller 1) (.directGrouping "spout" Constants/COORDINATED_STREAM_ID) (.directGrouping "count" Constants/COORDINATED_STREAM_ID)) (add-transactional-data data {0 [["dog"] ["cat"] ["apple"] ["dog"]]}) (bind topo-info (tracked-captured-topology cluster (.createTopology builder))) (submit-local-topology (:nimbus cluster) "transactional-test" {TOPOLOGY-MAX-SPOUT-PENDING 2 } (:topology topo-info)) (bind ack-tx! (fn [txid] (let [[to-ack not-to-ack] (separate #(-> % (.getValue 0) .getTransactionId (= txid)) @tuples)] (reset! tuples not-to-ack) (doseq [t to-ack] (ack! @collector t))))) (bind fail-tx! (fn [txid] (let [[to-fail not-to-fail] (separate #(-> % (.getValue 0) .getTransactionId (= txid)) @tuples)] (reset! tuples not-to-fail) (doseq [t to-fail] (fail! @collector t))))) ;; only check default streams (bind verify! (fn [expected] (let [results (-> topo-info :capturer .getResults)] (doseq [[component tuples] expected :let [emitted (->> (read-tuples results component "default") (map normalize-tx-tuple))]] (is (ms= tuples emitted))) (.clear results) ))) (tracked-wait topo-info 2) (verify! {"count" []}) (ack-tx! 1) (tracked-wait topo-info 1) (verify! {"count" [[1 "dog" 1] [1 "cat" 1]]}) (ack-tx! 2) (ack-tx! 1) (tracked-wait topo-info 2) (verify! {"count" [[2 "apple" 1] [2 "dog" 1]]}) )))) ================================================ FILE: storm-core/test/clj/backtype/storm/tuple_test.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.tuple-test (:use [clojure test]) (:import [backtype.storm.tuple Tuple]) (:use [backtype.storm testing])) (deftest test-lookup (let [ tuple (test-tuple [12 "hello"] :fields ["foo" "bar"]) ] (is (= 12 (tuple "foo"))) (is (= 12 (tuple :foo))) (is (= 12 (:foo tuple))) (is (= "hello" (:bar tuple))) (is (= :notfound (tuple "404" :notfound))))) (deftest test-indexed (let [ tuple (test-tuple [12 "hello"] :fields ["foo" "bar"]) ] (is (= 12 (nth tuple 0))) (is (= "hello" (nth tuple 1))))) (deftest test-seq (let [ tuple (test-tuple [12 "hello"] :fields ["foo" "bar"]) ] (is (= [["foo" 12] ["bar" "hello"]] (seq tuple))))) (deftest test-map (let [tuple (test-tuple [12 "hello"] :fields ["foo" "bar"]) ] (is (= {"foo" 42 "bar" "hello"} (.getMap (assoc tuple "foo" 42)))) (is (= {"foo" 42 "bar" "hello"} (.getMap (assoc tuple :foo 42)))) (is (= {"bar" "hello"} (.getMap (dissoc tuple "foo")))) (is (= {"bar" "hello"} (.getMap (dissoc tuple :foo)))) (is (= {"foo" 42 "bar" "world"} (.getMap (assoc (assoc tuple "foo" 42) :bar "world")))))) ================================================ FILE: storm-core/test/clj/backtype/storm/utils_test.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.utils-test (:import [backtype.storm Config]) (:import [backtype.storm.utils NimbusClient Utils]) (:import [com.netflix.curator.retry ExponentialBackoffRetry]) (:import [org.apache.thrift7.transport TTransportException]) (:use [backtype.storm config util]) (:use [clojure test]) ) (deftest test-new-curator-uses-exponential-backoff (let [expected_interval 2400 expected_retries 10 expected_ceiling (/ expected_interval 2) conf (merge (clojurify-structure (Utils/readDefaultConfig)) {Config/STORM_ZOOKEEPER_RETRY_INTERVAL expected_interval Config/STORM_ZOOKEEPER_RETRY_TIMES expected_retries Config/STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING expected_ceiling}) servers ["bogus_server"] arbitrary_port 42 curator (Utils/newCurator conf servers arbitrary_port) retry (-> curator .getZookeeperClient .getRetryPolicy) ] (is (.isAssignableFrom ExponentialBackoffRetry (.getClass retry))) (is (= (.getBaseSleepTimeMs retry) expected_interval)) (is (= (.getN retry) expected_retries)) (is (= (.getMaxRetryInterval retry) expected_ceiling)) (is (= (.getSleepTimeMs retry 10 0) expected_ceiling)) ) ) (deftest test-getConfiguredClient-throws-RunTimeException-on-bad-config (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin" Config/NIMBUS_HOST "" Config/NIMBUS_THRIFT_PORT 65535 })] (is (thrown? RuntimeException (NimbusClient/getConfiguredClient storm-conf))) ) ) (deftest test-getConfiguredClient-throws-RunTimeException-on-bad-args (let [storm-conf (read-storm-config)] (is (thrown? TTransportException (NimbusClient. storm-conf "" 65535) )) ) ) ================================================ FILE: storm-core/test/clj/backtype/storm/versioned_store_test.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.versioned-store-test (:use [clojure test]) (:use [backtype.storm testing]) (:import [backtype.storm.utils VersionedStore])) (defmacro defvstest [name [vs-sym] & body] `(deftest ~name (with-local-tmp [dir#] (let [~vs-sym (VersionedStore. dir#)] ~@body )))) (defvstest test-empty-version [vs] (let [v (.createVersion vs)] (.succeedVersion vs v) (is (= 1 (count (.getAllVersions vs)))) (is (= v (.mostRecentVersionPath vs))) )) (defvstest test-multiple-versions [vs] (.succeedVersion vs (.createVersion vs)) (Thread/sleep 100) (let [v (.createVersion vs)] (.succeedVersion vs v) (is (= 2 (count (.getAllVersions vs)))) (is (= v (.mostRecentVersionPath vs))) (.createVersion vs) (is (= v (.mostRecentVersionPath vs))) )) ================================================ FILE: storm-core/test/clj/storm/trident/integration_test.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 storm.trident.integration-test (:use [clojure test]) (:require [backtype.storm [testing :as t]]) (:import [storm.trident.testing Split CountAsAggregator StringLength TrueFilter MemoryMapState$Factory]) (:import [storm.trident.state StateSpec]) (:import [storm.trident.operation.impl CombinerAggStateUpdater]) (:use [storm.trident testing]) (:use [backtype.storm util])) (bootstrap-imports) (deftest test-memory-map-get-tuples (t/with-local-cluster [cluster] (with-drpc [drpc] (letlocals (bind topo (TridentTopology.)) (bind feeder (feeder-spout ["sentence"])) (bind word-counts (-> topo (.newStream "tester" feeder) (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word")) (.persistentAggregate (memory-map-state) (Count.) (fields "count")) (.parallelismHint 6) )) (-> topo (.newDRPCStream "all-tuples" drpc) (.broadcast) (.stateQuery word-counts (fields "args") (TupleCollectionGet.) (fields "word" "count")) (.project (fields "word" "count"))) (with-topology [cluster topo] (feed feeder [["hello the man said"] ["the"]]) (is (= #{["hello" 1] ["said" 1] ["the" 2] ["man" 1]} (into #{} (exec-drpc drpc "all-tuples" "man")))) (feed feeder [["the foo"]]) (is (= #{["hello" 1] ["said" 1] ["the" 3] ["man" 1] ["foo" 1]} (into #{} (exec-drpc drpc "all-tuples" "man"))))))))) (deftest test-word-count (t/with-local-cluster [cluster] (with-drpc [drpc] (letlocals (bind topo (TridentTopology.)) (bind feeder (feeder-spout ["sentence"])) (bind word-counts (-> topo (.newStream "tester" feeder) (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word")) (.persistentAggregate (memory-map-state) (Count.) (fields "count")) (.parallelismHint 6) )) (-> topo (.newDRPCStream "words" drpc) (.each (fields "args") (Split.) (fields "word")) (.groupBy (fields "word")) (.stateQuery word-counts (fields "word") (MapGet.) (fields "count")) (.aggregate (fields "count") (Sum.) (fields "sum")) (.project (fields "sum"))) (with-topology [cluster topo] (feed feeder [["hello the man said"] ["the"]]) (is (= [[2]] (exec-drpc drpc "words" "the"))) (is (= [[1]] (exec-drpc drpc "words" "hello"))) (feed feeder [["the man on the moon"] ["where are you"]]) (is (= [[4]] (exec-drpc drpc "words" "the"))) (is (= [[2]] (exec-drpc drpc "words" "man"))) (is (= [[8]] (exec-drpc drpc "words" "man where you the"))) ))))) ;; this test reproduces a bug where committer spouts freeze processing when ;; there's at least one repartitioning after the spout (deftest test-word-count-committer-spout (t/with-local-cluster [cluster] (with-drpc [drpc] (letlocals (bind topo (TridentTopology.)) (bind feeder (feeder-committer-spout ["sentence"])) (.setWaitToEmit feeder false) ;;this causes lots of empty batches (bind word-counts (-> topo (.newStream "tester" feeder) (.parallelismHint 2) (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word")) (.persistentAggregate (memory-map-state) (Count.) (fields "count")) (.parallelismHint 6) )) (-> topo (.newDRPCStream "words" drpc) (.each (fields "args") (Split.) (fields "word")) (.groupBy (fields "word")) (.stateQuery word-counts (fields "word") (MapGet.) (fields "count")) (.aggregate (fields "count") (Sum.) (fields "sum")) (.project (fields "sum"))) (with-topology [cluster topo] (feed feeder [["hello the man said"] ["the"]]) (is (= [[2]] (exec-drpc drpc "words" "the"))) (is (= [[1]] (exec-drpc drpc "words" "hello"))) (Thread/sleep 1000) ;; this is necessary to reproduce the bug where committer spouts freeze processing (feed feeder [["the man on the moon"] ["where are you"]]) (is (= [[4]] (exec-drpc drpc "words" "the"))) (is (= [[2]] (exec-drpc drpc "words" "man"))) (is (= [[8]] (exec-drpc drpc "words" "man where you the"))) (feed feeder [["the the"]]) (is (= [[6]] (exec-drpc drpc "words" "the"))) (feed feeder [["the"]]) (is (= [[7]] (exec-drpc drpc "words" "the"))) ))))) (deftest test-count-agg (t/with-local-cluster [cluster] (with-drpc [drpc] (letlocals (bind topo (TridentTopology.)) (-> topo (.newDRPCStream "numwords" drpc) (.each (fields "args") (Split.) (fields "word")) (.aggregate (CountAsAggregator.) (fields "count")) (.parallelismHint 2) ;;this makes sure batchGlobal is working correctly (.project (fields "count"))) (with-topology [cluster topo] (doseq [i (range 100)] (is (= [[1]] (exec-drpc drpc "numwords" "the")))) (is (= [[0]] (exec-drpc drpc "numwords" ""))) (is (= [[8]] (exec-drpc drpc "numwords" "1 2 3 4 5 6 7 8"))) ))))) (deftest test-split-merge (t/with-local-cluster [cluster] (with-drpc [drpc] (letlocals (bind topo (TridentTopology.)) (bind drpc-stream (-> topo (.newDRPCStream "splitter" drpc))) (bind s1 (-> drpc-stream (.each (fields "args") (Split.) (fields "word")) (.project (fields "word")))) (bind s2 (-> drpc-stream (.each (fields "args") (StringLength.) (fields "len")) (.project (fields "len")))) (.merge topo [s1 s2]) (with-topology [cluster topo] (is (t/ms= [[7] ["the"] ["man"]] (exec-drpc drpc "splitter" "the man"))) (is (t/ms= [[5] ["hello"]] (exec-drpc drpc "splitter" "hello"))) ))))) (deftest test-multiple-groupings-same-stream (t/with-local-cluster [cluster] (with-drpc [drpc] (letlocals (bind topo (TridentTopology.)) (bind drpc-stream (-> topo (.newDRPCStream "tester" drpc) (.each (fields "args") (TrueFilter.)))) (bind s1 (-> drpc-stream (.groupBy (fields "args")) (.aggregate (CountAsAggregator.) (fields "count")))) (bind s2 (-> drpc-stream (.groupBy (fields "args")) (.aggregate (CountAsAggregator.) (fields "count")))) (.merge topo [s1 s2]) (with-topology [cluster topo] (is (t/ms= [["the" 1] ["the" 1]] (exec-drpc drpc "tester" "the"))) (is (t/ms= [["aaaaa" 1] ["aaaaa" 1]] (exec-drpc drpc "tester" "aaaaa"))) ))))) (deftest test-multi-repartition (t/with-local-cluster [cluster] (with-drpc [drpc] (letlocals (bind topo (TridentTopology.)) (bind drpc-stream (-> topo (.newDRPCStream "tester" drpc) (.each (fields "args") (Split.) (fields "word")) (.shuffle) (.shuffle) (.aggregate (CountAsAggregator.) (fields "count")) )) (with-topology [cluster topo] (is (t/ms= [[2]] (exec-drpc drpc "tester" "the man"))) (is (t/ms= [[1]] (exec-drpc drpc "tester" "aaa"))) ))))) (deftest test-stream-projection-validation (t/with-local-cluster [cluster] (letlocals (bind feeder (feeder-committer-spout ["sentence"])) (bind topo (TridentTopology.)) ;; valid projection fields will not throw exceptions (bind word-counts (-> topo (.newStream "tester" feeder) (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word")) (.persistentAggregate (memory-map-state) (Count.) (fields "count")) (.parallelismHint 6) )) (bind stream (-> topo (.newStream "tester" feeder))) ;; test .each (is (thrown? IllegalArgumentException (-> stream (.each (fields "sentence1") (Split.) (fields "word"))))) ;; test .groupBy (is (thrown? IllegalArgumentException (-> stream (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word1"))))) ;; test .aggregate (is (thrown? IllegalArgumentException (-> stream (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word")) (.aggregate (fields "word1") (Count.) (fields "count"))))) ;; test .project (is (thrown? IllegalArgumentException (-> stream (.project (fields "sentence1"))))) ;; test .partitionBy (is (thrown? IllegalArgumentException (-> stream (.partitionBy (fields "sentence1"))))) ;; test .partitionAggregate (is (thrown? IllegalArgumentException (-> stream (.each (fields "sentence") (Split.) (fields "word")) (.partitionAggregate (fields "word1") (Count.) (fields "count"))))) ;; test .persistentAggregate (is (thrown? IllegalArgumentException (-> stream (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word")) (.persistentAggregate (StateSpec. (MemoryMapState$Factory.)) (fields "non-existent") (Count.) (fields "count"))))) ;; test .partitionPersist (is (thrown? IllegalArgumentException (-> stream (.each (fields "sentence") (Split.) (fields "word")) (.groupBy (fields "word")) (.partitionPersist (StateSpec. (MemoryMapState$Factory.)) (fields "non-existent") (CombinerAggStateUpdater. (Count.)) (fields "count"))))) ;; test .stateQuery (with-drpc [drpc] (is (thrown? IllegalArgumentException (-> topo (.newDRPCStream "words" drpc) (.each (fields "args") (Split.) (fields "word")) (.groupBy (fields "word")) (.stateQuery word-counts (fields "word1") (MapGet.) (fields "count")))))) ))) ;; (deftest test-split-merge ;; (t/with-local-cluster [cluster] ;; (with-drpc [drpc] ;; (letlocals ;; (bind topo (TridentTopology.)) ;; (bind drpc-stream (-> topo (.newDRPCStream "splitter" drpc))) ;; (bind s1 ;; (-> drpc-stream ;; (.each (fields "args") (Split.) (fields "word")) ;; (.project (fields "word")))) ;; (bind s2 ;; (-> drpc-stream ;; (.each (fields "args") (StringLength.) (fields "len")) ;; (.project (fields "len")))) ;; ;; (.merge topo [s1 s2]) ;; (with-topology [cluster topo] ;; (is (t/ms= [[7] ["the"] ["man"]] (exec-drpc drpc "splitter" "the man"))) ;; (is (t/ms= [[5] ["hello"]] (exec-drpc drpc "splitter" "hello"))) ;; ))))) ================================================ FILE: storm-core/test/clj/storm/trident/state_test.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 storm.trident.state-test (:use [clojure test]) (:require [backtype.storm [testing :as t]]) (:import [storm.trident.operation.builtin Count]) (:import [storm.trident.state OpaqueValue]) (:import [storm.trident.state CombinerValueUpdater]) (:import [storm.trident.state.map TransactionalMap OpaqueMap]) (:import [storm.trident.testing MemoryBackingMap]) (:use [storm.trident testing]) (:use [backtype.storm util])) (defn single-get [map key] (-> map (.multiGet [[key]]) first)) (defn single-update [map key amt] (-> map (.multiUpdate [[key]] [(CombinerValueUpdater. (Count.) amt)]) first)) (deftest test-opaque-value (let [opqval (OpaqueValue. 8 "v1" "v0") upval0 (.update opqval 8 "v2") upval1 (.update opqval 9 "v2") ] (is (= "v1" (.get opqval nil))) (is (= "v1" (.get opqval 100))) (is (= "v1" (.get opqval 9))) (is (= "v0" (.get opqval 8))) (let [has-exception (try (.get opqval 7) false (catch Exception e true))] (is (= true has-exception))) (is (= "v0" (.getPrev opqval))) (is (= "v1" (.getCurr opqval))) ;; update with current (is (= "v0" (.getPrev upval0))) (is (= "v2" (.getCurr upval0))) (not (identical? opqval upval0)) ;; update (is (= "v1" (.getPrev upval1))) (is (= "v2" (.getCurr upval1))) (not (identical? opqval upval1)) )) (deftest test-opaque-map (let [map (OpaqueMap/build (MemoryBackingMap.))] (.beginCommit map 1) (is (= nil (single-get map "a"))) ;; tests that intra-batch caching works (is (= 1 (single-update map "a" 1))) (is (= 3 (single-update map "a" 2))) (.commit map 1) (.beginCommit map 1) (is (= nil (single-get map "a"))) (is (= 2 (single-update map "a" 2))) (.commit map 1) (.beginCommit map 2) (is (= 2 (single-get map "a"))) (is (= 5 (single-update map "a" 3))) (is (= 6 (single-update map "a" 1))) (.commit map 2) )) (deftest test-transactional-map (let [map (TransactionalMap/build (MemoryBackingMap.))] (.beginCommit map 1) (is (= nil (single-get map "a"))) ;; tests that intra-batch caching works (is (= 1 (single-update map "a" 1))) (is (= 3 (single-update map "a" 2))) (.commit map 1) (.beginCommit map 1) (is (= 3 (single-get map "a"))) ;; tests that intra-batch caching has no effect if it's the same commit as previous commit (is (= 3 (single-update map "a" 1))) (is (= 3 (single-update map "a" 2))) (.commit map 1) (.beginCommit map 2) (is (= 3 (single-get map "a"))) (is (= 6 (single-update map "a" 3))) (is (= 7 (single-update map "a" 1))) (.commit map 2) )) ================================================ FILE: storm-core/test/clj/storm/trident/tuple_test.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 storm.trident.tuple-test (:use [clojure test]) (:require [backtype.storm [testing :as t]]) (:import [storm.trident.tuple TridentTupleView TridentTupleView$ProjectionFactory TridentTupleView$FreshOutputFactory TridentTupleView$OperationOutputFactory TridentTupleView$RootFactory]) (:use [storm.trident testing]) (:use [backtype.storm util])) (deftest test-fresh (letlocals (bind fresh-factory (TridentTupleView$FreshOutputFactory. (fields "a" "b" "c"))) (bind tt (.create fresh-factory [3 2 1])) (is (= [3 2 1] tt)) (is (= 3 (.getValueByField tt "a"))) (is (= 2 (.getValueByField tt "b"))) (is (= 1 (.getValueByField tt "c"))) )) (deftest test-projection (letlocals (bind fresh-factory (TridentTupleView$FreshOutputFactory. (fields "a" "b" "c" "d" "e"))) (bind project-factory (TridentTupleView$ProjectionFactory. fresh-factory (fields "d" "a"))) (bind tt (.create fresh-factory [3 2 1 4 5])) (bind tt2 (.create fresh-factory [9 8 7 6 10])) (bind pt (.create project-factory tt)) (bind pt2 (.create project-factory tt2)) (is (= [4 3] pt)) (is (= [6 9] pt2)) (is (= 4 (.getValueByField pt "d"))) (is (= 3 (.getValueByField pt "a"))) (is (= 6 (.getValueByField pt2 "d"))) (is (= 9 (.getValueByField pt2 "a"))) )) (deftest test-appends (letlocals (bind fresh-factory (TridentTupleView$FreshOutputFactory. (fields "a" "b" "c"))) (bind append-factory (TridentTupleView$OperationOutputFactory. fresh-factory (fields "d" "e"))) (bind append-factory2 (TridentTupleView$OperationOutputFactory. append-factory (fields "f"))) (bind tt (.create fresh-factory [1 2 3])) (bind tt2 (.create append-factory tt [4 5])) (bind tt3 (.create append-factory2 tt2 [7])) (is (= [1 2 3 4 5 7] tt3)) (is (= 5 (.getValueByField tt2 "e"))) (is (= 5 (.getValueByField tt3 "e"))) (is (= 7 (.getValueByField tt3 "f"))) )) (deftest test-root (letlocals (bind root-factory (TridentTupleView$RootFactory. (fields "a" "b"))) (bind storm-tuple (t/test-tuple ["a" 1])) (bind tt (.create root-factory storm-tuple)) (is (= ["a" 1] tt)) (is (= "a" (.getValueByField tt "a"))) (is (= 1 (.getValueByField tt "b"))) (bind append-factory (TridentTupleView$OperationOutputFactory. root-factory (fields "c"))) (bind tt2 (.create append-factory tt [3])) (is (= ["a" 1 3] tt2)) (is (= "a" (.getValueByField tt2 "a"))) (is (= 1 (.getValueByField tt2 "b"))) (is (= 3 (.getValueByField tt2 "c"))) )) (deftest test-complex (letlocals (bind fresh-factory (TridentTupleView$FreshOutputFactory. (fields "a" "b" "c"))) (bind append-factory1 (TridentTupleView$OperationOutputFactory. fresh-factory (fields "d"))) (bind append-factory2 (TridentTupleView$OperationOutputFactory. append-factory1 (fields "e" "f"))) (bind project-factory1 (TridentTupleView$ProjectionFactory. append-factory2 (fields "a" "f" "b"))) (bind append-factory3 (TridentTupleView$OperationOutputFactory. project-factory1 (fields "c"))) (bind tt (.create fresh-factory [1 2 3])) (bind tt2 (.create append-factory1 tt [4])) (bind tt3 (.create append-factory2 tt2 [5 6])) (bind tt4 (.create project-factory1 tt3)) (bind tt5 (.create append-factory3 tt4 [8])) (is (= [1 2 3] tt)) (is (= [1 2 3 4] tt2)) (is (= [1 2 3 4 5 6] tt3)) (is (= [1 6 2] tt4)) (is (= [1 6 2 8] tt5)) (is (= 1 (.getValueByField tt5 "a"))) (is (= 6 (.getValueByField tt5 "f"))) (is (= 2 (.getValueByField tt5 "b"))) (is (= 8 (.getValueByField tt5 "c"))) )) ================================================ FILE: storm-core/test/clj/zilch/test/mq.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 zilch.test.mq (:use clojure.test) (:import [java.util Arrays UUID]) (:require [zilch.mq :as mq])) (defn uuid [] (str (UUID/randomUUID))) (defn random-msg [] (byte-array (map byte (for [i (range (Integer. (int (rand 100))))] (Integer. (int (rand 100))) )))) (def url (str "inproc://" (uuid)) ;; (str "ipc://" (uuid)) ;; (str "tcp://127.0.0.1:" (+ 4000 (Math/round (rand 1000))))) ) (deftest zilch (testing "zilch" (testing "should be able to" (testing "push / pull" (mq/with-context context 2 (with-open [s0 (-> context (mq/socket mq/pull) (mq/bind url)) s1 (-> context (mq/socket mq/push) (mq/connect url))] (let [msg (random-msg) push (future (mq/send s1 msg)) pull (future (mq/recv s0))] (is (Arrays/equals msg @pull)))))) (testing "pub / sub" (mq/with-context context 2 (with-open [s0 (-> context (mq/socket mq/pub) (mq/bind url)) s1 (-> context (mq/socket mq/sub) (mq/subscribe) (mq/connect url))] (let [msg (random-msg) pub (future (mq/send s0 msg)) sub (future (mq/recv s1))] (is (Arrays/equals msg @sub)))))) (testing "pair / pair" (mq/with-context context 2 (with-open [s0 (-> context (mq/socket mq/pair) (mq/bind url)) s1 (-> context (mq/socket mq/pair) (mq/connect url))] (let [msg0 (random-msg) pair0 (future (mq/send s0 msg0) (mq/recv s0)) msg1 (random-msg) pair1 (future (mq/send s1 msg1) (mq/recv s1))] (is (Arrays/equals msg1 @pair0)) (is (Arrays/equals msg0 @pair1)))))) (testing "req / rep" (mq/with-context context 2 (with-open [s0 (-> context (mq/socket mq/rep) (mq/bind url)) s1 (-> context (mq/socket mq/req) (mq/connect url))] (let [msg (random-msg) req (future (mq/send s1 msg) (mq/recv s1)) rep (future (mq/recv s0) (mq/send s0 msg))] (is (Arrays/equals msg @req)))))) (testing "req / xrep") (testing "xreq / rep") (testing "xreq / xrep")))) ================================================ FILE: storm-core/test/multilang/fy/bolt.fy ================================================ require: "mocks" class TestBolt : Storm Bolt { def process: tuple { emit: $ [tuple values join: ", "] ack: tuple } } FancySpec describe: Storm Bolt with: { before_each: { Storm Protocol Input clear Storm Protocol Output clear @storm = Storm Protocol new @in = Storm Protocol Input @out = Storm Protocol Output } it: "runs as as expected" for: 'run when: { conf = <['some_conf => false]> context = <['some_context => true]> tup1 = <['id => 1, 'comp => 2, 'stream => 3, 'task => 4, 'tuple => [1,2,3,4]]> task_ids_1 = <['task_ids => [1,2,3,4]]> # part of the protocol, random values though tup2 = <['id => 2, 'comp => 3, 'stream => 4, 'task => 5, 'tuple => ["hello", "world"]]> task_ids_2 = <['task_ids => [2,3,4,5]]> # same here @in input: [ "/tmp/", conf to_json() , context to_json(), # tuples: tup1 to_json(), task_ids_1 to_json(), tup2 to_json(), task_ids_2 to_json() ] b = TestBolt new b run @out sent select: |m| { m includes?: $ tup1['tuple] join: ", " } size is == 1 @out sent select: |m| { m includes?: $ tup2['tuple] join: ", " } size is == 1 } } ================================================ FILE: storm-core/test/multilang/fy/mocks.fy ================================================ require: "../../../src/multilang/fy/storm" class MockedIO { def initialize { @out = [] @in = [] } def print: string { @out << (string to_s) } def println: string { @out << (string ++ "\n") } def input: input { input each: |i| { @in << (i ++ "\n") @in << "end\n" } } def readline { if: (@in empty?) then: { "No input left" raise! } @in shift } def receive_msg: msg { @in << (msg ++ "\n") @in << "end\n" } def clear { @in = [] @out = [] } def flush { } def received { @in } def sent { @out } } class Storm Protocol { Input = MockedIO new Output = MockedIO new } ================================================ FILE: storm-core/test/multilang/fy/protocol.fy ================================================ require: "mocks" FancySpec describe: Storm Protocol with: { before_each: { Storm Protocol Input clear Storm Protocol Output clear @storm = Storm Protocol new @in = Storm Protocol Input @out = Storm Protocol Output @tuple = Storm Tuple new: 1 component: 2 stream: 3 task: 4 values: [1,2,3,4] } it: "reads a string message correctly" for: 'read_string_message when: { @in receive_msg: "/tmp/" @storm read_string_message is == "/tmp/" } it: "reads a json message correctly" for: 'read_message when: { @in receive_msg: "{\"foo\":123, \"bar\":\"foobar\", \"tuple\":[1,2,\"cool\"]}" msg = @storm read_message msg is == <["foo" => 123, "bar" => "foobar", "tuple" => [1,2,"cool"]]> } it: "sends a message correctly" for: 'send: when: { msg = <['hello => "world", 'testing => 42]> @storm send: msg @out sent is == ["#{msg to_json()}\n", "end\n"] } it: "sends the pid to the parent process" for: 'send_pid: when: { @storm send_pid: "/tmp/" pid = Process pid() @out sent is == ["#{pid}\n"] } it: "syncs with the parent process" for: 'sync when: { @storm sync @out sent is == ["sync\n"] } it: "emits a tuple to storm" for: 'emit_tuple:stream:anchors:direct: when: { tuple_values = ["hello", "world"] @storm emit_tuple: tuple_values emit_msg = JSON parse(@out sent[-2]) # last one is "end" emit_msg is == <["command" => "emit", "anchors" => [], "tuple" => tuple_values]> } it: "acks a tuple" for: 'ack: when: { @storm ack: @tuple ack_msg = JSON parse(@out sent[-2]) ack_msg is == <["command" => "ack", "id" => @tuple id]> } it: "fails a tuple" for: 'fail: when: { @storm fail: @tuple fail_msg = JSON parse(@out sent[-2]) fail_msg is == <["command" => "fail", "id" => @tuple id]> } it: "logs a message" for: 'log: when: { @storm log: "log test" log_msg = JSON parse(@out sent[-2]) log_msg is == <["command" => "log", "msg" => "log test"]> } } ================================================ FILE: storm-lib/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))) (def MODULES (-> ROOT-DIR (str "/../MODULES") slurp (.split "\n") (#(filter (fn [m] (not= m "storm-console-logging")) %)) )) (def DEPENDENCIES (for [m MODULES] [(symbol (str "storm/" m)) VERSION])) ;; for lib pom.xml, change the symbol to storm/storm-liba and filter out storm-console-logging from modules (eval `(defproject storm/storm-lib ~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] :min-lein-version "2.0.0" :target-path "target" )) ================================================ FILE: storm-netty/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))) (eval `(defproject storm/storm-netty ~VERSION :dependencies [[storm/storm-core ~VERSION] [io.netty/netty "3.6.3.Final"]] :java-source-paths ["src/jvm"] :test-paths ["test/clj"] :profiles {:release {}} :jvm-opts ["-Djava.library.path=/usr/local/lib:/opt/local/lib:/usr/lib"] :target-path "target" :javac-options ["-target" "1.6" "-source" "1.6"] :aot :all)) ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/Client.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import java.net.InetSocketAddress; import java.util.Map; import java.util.Random; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.jboss.netty.bootstrap.ClientBootstrap; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.Config; import backtype.storm.messaging.IConnection; import backtype.storm.messaging.TaskMessage; import backtype.storm.utils.Utils; class Client implements IConnection { private static final Logger LOG = LoggerFactory.getLogger(Client.class); private final int max_retries; private final int base_sleep_ms; private final int max_sleep_ms; private LinkedBlockingQueue message_queue; //entry should either be TaskMessage or ControlMessage private AtomicReference channelRef; private final ClientBootstrap bootstrap; private InetSocketAddress remote_addr; private AtomicInteger retries; private final Random random = new Random(); private final ChannelFactory factory; private final int buffer_size; private final AtomicBoolean being_closed; @SuppressWarnings("rawtypes") Client(Map storm_conf, String host, int port) { message_queue = new LinkedBlockingQueue(); retries = new AtomicInteger(0); channelRef = new AtomicReference(null); being_closed = new AtomicBoolean(false); // Configure buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); max_retries = Math.min(30, Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES))); base_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); max_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS)); if (maxWorkers > 0) { factory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), maxWorkers); } else { factory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()); } bootstrap = new ClientBootstrap(factory); bootstrap.setOption("tcpNoDelay", true); bootstrap.setOption("sendBufferSize", buffer_size); bootstrap.setOption("keepAlive", true); // Set up the pipeline factory. bootstrap.setPipelineFactory(new StormClientPipelineFactory(this)); // Start the connection attempt. remote_addr = new InetSocketAddress(host, port); bootstrap.connect(remote_addr); } /** * We will retry connection with exponential back-off policy */ void reconnect() { try { int tried_count = retries.incrementAndGet(); if (tried_count <= max_retries) { Thread.sleep(getSleepTimeMs()); LOG.info("Reconnect ... [{}]", tried_count); bootstrap.connect(remote_addr); LOG.debug("connection started..."); } else { LOG.warn("Remote address is not reachable. We will close this client."); close(); } } catch (InterruptedException e) { LOG.warn("connection failed", e); } } /** * # of milliseconds to wait per exponential back-off policy */ private int getSleepTimeMs() { int backoff = 1 << retries.get(); int sleepMs = base_sleep_ms * Math.max(1, random.nextInt(backoff)); if ( sleepMs > max_sleep_ms ) sleepMs = max_sleep_ms; return sleepMs; } /** * Enqueue a task message to be sent to server */ public void send(int task, byte[] message) { //throw exception if the client is being closed if (being_closed.get()) { throw new RuntimeException("Client is being closed, and does not take requests any more"); } try { message_queue.put(new TaskMessage(task, message)); } catch (InterruptedException e) { throw new RuntimeException(e); } } /** * Take all enqueued messages from queue * @return * @throws InterruptedException */ MessageBatch takeMessages() throws InterruptedException { //1st message MessageBatch batch = new MessageBatch(buffer_size); Object msg = message_queue.take(); batch.add(msg); //we will discard any message after CLOSE if (msg==ControlMessage.CLOSE_MESSAGE) return batch; while (!batch.isFull()) { //peek the next message msg = message_queue.peek(); //no more messages if (msg == null) break; //we will discard any message after CLOSE if (msg==ControlMessage.CLOSE_MESSAGE) { message_queue.take(); batch.add(msg); break; } //try to add this msg into batch if (!batch.tryAdd((TaskMessage) msg)) break; //remove this message message_queue.take(); } return batch; } /** * gracefully close this client. * * We will send all existing requests, and then invoke close_n_release() method */ public synchronized void close() { if (!being_closed.get()) { //enqueue a CLOSE message so that shutdown() will be invoked try { message_queue.put(ControlMessage.CLOSE_MESSAGE); being_closed.set(true); } catch (InterruptedException e) { close_n_release(); } } } /** * close_n_release() is invoked after all messages have been sent. */ void close_n_release() { if (channelRef.get() != null) channelRef.get().close().awaitUninterruptibly(); //we need to release resources new Thread(new Runnable() { @Override public void run() { factory.releaseExternalResources(); }}).start(); } public TaskMessage recv(int flags) { throw new RuntimeException("Client connection should not receive any messages"); } void setChannel(Channel channel) { channelRef.set(channel); //reset retries if (channel != null) retries.set(0); } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/Context.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import java.util.Map; import java.util.Vector; import backtype.storm.messaging.IConnection; import backtype.storm.messaging.IContext; public class Context implements IContext { @SuppressWarnings("rawtypes") private Map storm_conf; private volatile Vector connections; /** * initialization per Storm configuration */ @SuppressWarnings("rawtypes") public void prepare(Map storm_conf) { this.storm_conf = storm_conf; connections = new Vector(); } /** * establish a server with a binding port */ public IConnection bind(String storm_id, int port) { IConnection server = new Server(storm_conf, port); connections.add(server); return server; } /** * establish a connection to a remote server */ public IConnection connect(String storm_id, String host, int port) { IConnection client = new Client(storm_conf, host, port); connections.add(client); return client; } /** * terminate this context */ public void term() { for (IConnection conn : connections) { conn.close(); } connections = null; } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/ControlMessage.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBufferOutputStream; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.channel.Channel; enum ControlMessage { CLOSE_MESSAGE((short)-100), EOB_MESSAGE((short)-201), OK_RESPONSE((short)-200), FAILURE_RESPONSE((short)-400); private short code; //private constructor private ControlMessage(short code) { this.code = code; } /** * Return a control message per an encoded status code * @param encoded * @return */ static ControlMessage mkMessage(short encoded) { for(ControlMessage cm: ControlMessage.values()) { if(encoded == cm.code) return cm; } return null; } int encodeLength() { return 2; //short } /** * encode the current Control Message into a channel buffer * @throws Exception */ ChannelBuffer buffer() throws Exception { ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encodeLength())); write(bout); bout.close(); return bout.buffer(); } void write(ChannelBufferOutputStream bout) throws Exception { bout.writeShort(code); } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/MessageBatch.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import java.util.ArrayList; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBufferOutputStream; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.channel.Channel; import backtype.storm.messaging.TaskMessage; class MessageBatch { private int buffer_size; private ArrayList msgs; private int encoded_length; MessageBatch(int buffer_size) { this.buffer_size = buffer_size; msgs = new ArrayList(); encoded_length = ControlMessage.EOB_MESSAGE.encodeLength(); } void add(Object obj) { if (obj == null) throw new RuntimeException("null object forbidded in message batch"); if (obj instanceof TaskMessage) { TaskMessage msg = (TaskMessage)obj; msgs.add(msg); encoded_length += msgEncodeLength(msg); return; } if (obj instanceof ControlMessage) { ControlMessage msg = (ControlMessage)obj; msgs.add(msg); encoded_length += msg.encodeLength(); return; } throw new RuntimeException("Unsuppoted object type "+obj.getClass().getName()); } void remove(Object obj) { if (obj == null) return; if (obj instanceof TaskMessage) { TaskMessage msg = (TaskMessage)obj; msgs.remove(msg); encoded_length -= msgEncodeLength(msg); return; } if (obj instanceof ControlMessage) { ControlMessage msg = (ControlMessage)obj; msgs.remove(msg); encoded_length -= msg.encodeLength(); return; } } Object get(int index) { return msgs.get(index); } /** * try to add a TaskMessage to a batch * @param taskMsg * @return false if the msg could not be added due to buffer size limit; true otherwise */ boolean tryAdd(TaskMessage taskMsg) { if ((encoded_length + msgEncodeLength(taskMsg)) > buffer_size) return false; add(taskMsg); return true; } private int msgEncodeLength(TaskMessage taskMsg) { if (taskMsg == null) return 0; int size = 6; //INT + SHORT if (taskMsg.message() != null) size += taskMsg.message().length; return size; } /** * Has this batch used up allowed buffer size * @return */ boolean isFull() { return encoded_length >= buffer_size; } /** * true if this batch doesn't have any messages * @return */ boolean isEmpty() { return msgs.isEmpty(); } /** * # of msgs in this batch * @return */ int size() { return msgs.size(); } /** * create a buffer containing the encoding of this batch */ ChannelBuffer buffer() throws Exception { ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encoded_length)); for (Object msg : msgs) if (msg instanceof TaskMessage) writeTaskMessage(bout, (TaskMessage)msg); else ((ControlMessage)msg).write(bout); //add a END_OF_BATCH indicator ControlMessage.EOB_MESSAGE.write(bout); bout.close(); return bout.buffer(); } /** * write a TaskMessage into a stream * * Each TaskMessage is encoded as: * task ... short(2) * len ... int(4) * payload ... byte[] * */ private void writeTaskMessage(ChannelBufferOutputStream bout, TaskMessage message) throws Exception { int payload_len = 0; if (message.message() != null) payload_len = message.message().length; int task_id = message.task(); if (task_id > Short.MAX_VALUE) throw new RuntimeException("Task ID should not exceed "+Short.MAX_VALUE); bout.writeShort((short)task_id); bout.writeInt(payload_len); if (payload_len >0) bout.write(message.message()); } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.handler.codec.frame.FrameDecoder; import backtype.storm.messaging.TaskMessage; public class MessageDecoder extends FrameDecoder { /* * Each ControlMessage is encoded as: * code (<0) ... short(2) * Each TaskMessage is encoded as: * task (>=0) ... short(2) * len ... int(4) * payload ... byte[] * */ protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buf) throws Exception { // Make sure that we have received at least a short if (buf.readableBytes() < 2) { //need more data return null; } // Mark the current buffer position before reading task/len field // because the whole frame might not be in the buffer yet. // We will reset the buffer position to the marked position if // there's not enough bytes in the buffer. buf.markReaderIndex(); //read the short field short code = buf.readShort(); //case 1: Control message ControlMessage ctrl_msg = ControlMessage.mkMessage(code); if (ctrl_msg != null) return ctrl_msg; //case 2: task Message short task = code; // Make sure that we have received at least an integer (length) if (buf.readableBytes() < 4) { //need more data buf.resetReaderIndex(); return null; } // Read the length field. int length = buf.readInt(); if (length<=0) { return new TaskMessage(task, null); } // Make sure if there's enough bytes in the buffer. if (buf.readableBytes() < length) { // The whole bytes were not received yet - return null. buf.resetReaderIndex(); return null; } // There's enough bytes in the buffer. Read it. ChannelBuffer payload = buf.readBytes(length); // Successfully decoded a frame. // Return a TaskMessage object return new TaskMessage(task,payload.array()); } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/MessageEncoder.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.handler.codec.oneone.OneToOneEncoder; public class MessageEncoder extends OneToOneEncoder { @Override protected Object encode(ChannelHandlerContext ctx, Channel channel, Object obj) throws Exception { if (obj instanceof ControlMessage) { return ((ControlMessage)obj).buffer(); } if (obj instanceof MessageBatch) { return ((MessageBatch)obj).buffer(); } throw new RuntimeException("Unsupported encoding of object of class "+obj.getClass().getName()); } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/Server.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import java.net.InetSocketAddress; import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.Executors; import org.jboss.netty.bootstrap.ServerBootstrap; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; import org.jboss.netty.channel.group.ChannelGroup; import org.jboss.netty.channel.group.DefaultChannelGroup; import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.Config; import backtype.storm.messaging.IConnection; import backtype.storm.messaging.TaskMessage; import backtype.storm.utils.Utils; class Server implements IConnection { private static final Logger LOG = LoggerFactory.getLogger(Server.class); @SuppressWarnings("rawtypes") Map storm_conf; int port; private LinkedBlockingQueue message_queue; volatile ChannelGroup allChannels = new DefaultChannelGroup("storm-server"); final ChannelFactory factory; final ServerBootstrap bootstrap; @SuppressWarnings("rawtypes") Server(Map storm_conf, int port) { this.storm_conf = storm_conf; this.port = port; message_queue = new LinkedBlockingQueue(); // Configure the server. int buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS)); if (maxWorkers > 0) { factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), maxWorkers); } else { factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()); } bootstrap = new ServerBootstrap(factory); bootstrap.setOption("child.tcpNoDelay", true); bootstrap.setOption("child.receiveBufferSize", buffer_size); bootstrap.setOption("child.keepAlive", true); // Set up the pipeline factory. bootstrap.setPipelineFactory(new StormServerPipelineFactory(this)); // Bind and start to accept incoming connections. Channel channel = bootstrap.bind(new InetSocketAddress(port)); allChannels.add(channel); } /** * enqueue a received message * @param message * @throws InterruptedException */ protected void enqueue(TaskMessage message) throws InterruptedException { message_queue.put(message); LOG.debug("message received with task: {}, payload size: {}", message.task(), message.message().length); } /** * fetch a message from message queue synchronously (flags != 1) or asynchronously (flags==1) */ public TaskMessage recv(int flags) { if ((flags & 0x01) == 0x01) { //non-blocking return message_queue.poll(); } else { try { TaskMessage request = message_queue.take(); LOG.debug("request to be processed: {}", request); return request; } catch (InterruptedException e) { LOG.info("exception within msg receiving", e); return null; } } } /** * register a newly created channel * @param channel */ protected void addChannel(Channel channel) { allChannels.add(channel); } /** * close a channel * @param channel */ protected void closeChannel(Channel channel) { channel.close().awaitUninterruptibly(); allChannels.remove(channel); } /** * close all channels, and release resources */ public synchronized void close() { if (allChannels != null) { allChannels.close().awaitUninterruptibly(); factory.releaseExternalResources(); allChannels = null; } } public void send(int task, byte[] message) { throw new RuntimeException("Server connection should not send any messages"); } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import java.net.ConnectException; import java.util.concurrent.atomic.AtomicBoolean; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFuture; import org.jboss.netty.channel.ChannelFutureListener; import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.channel.ChannelStateEvent; import org.jboss.netty.channel.ExceptionEvent; import org.jboss.netty.channel.MessageEvent; import org.jboss.netty.channel.SimpleChannelUpstreamHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class StormClientHandler extends SimpleChannelUpstreamHandler { private static final Logger LOG = LoggerFactory.getLogger(StormClientHandler.class); private Client client; private AtomicBoolean being_closed; long start_time; StormClientHandler(Client client) { this.client = client; being_closed = new AtomicBoolean(false); start_time = System.currentTimeMillis(); } @Override public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent event) { //register the newly established channel Channel channel = event.getChannel(); client.setChannel(channel); LOG.debug("connection established to a remote host"); //send next request try { sendRequests(channel, client.takeMessages()); } catch (InterruptedException e) { channel.close(); } } @Override public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) { LOG.debug("send/recv time (ms): {}", (System.currentTimeMillis() - start_time)); //examine the response message from server ControlMessage msg = (ControlMessage)event.getMessage(); if (msg==ControlMessage.FAILURE_RESPONSE) LOG.info("failure response:{}", msg); //send next request Channel channel = event.getChannel(); try { sendRequests(channel, client.takeMessages()); } catch (InterruptedException e) { channel.close(); } } /** * Retrieve a request from message queue, and send to server * @param channel */ private void sendRequests(Channel channel, final MessageBatch requests) { if (requests==null || requests.size()==0 || being_closed.get()) return; //if task==CLOSE_MESSAGE for our last request, the channel is to be closed Object last_msg = requests.get(requests.size()-1); if (last_msg==ControlMessage.CLOSE_MESSAGE) { being_closed.set(true); requests.remove(last_msg); } //we may don't need do anything if no requests found if (requests.isEmpty()) { if (being_closed.get()) client.close_n_release(); return; } //write request into socket channel ChannelFuture future = channel.write(requests); future.addListener(new ChannelFutureListener() { public void operationComplete(ChannelFuture future) throws Exception { if (!future.isSuccess()) { LOG.info("failed to send requests:", future.getCause()); future.getChannel().close(); } else { LOG.debug("{} request(s) sent", requests.size()); } if (being_closed.get()) client.close_n_release(); } }); } @Override public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) { Throwable cause = event.getCause(); if (!(cause instanceof ConnectException)) { LOG.info("Connection failed:", cause); } if (!being_closed.get()) { client.setChannel(null); client.reconnect(); } } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import org.jboss.netty.channel.ChannelPipeline; import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.Channels; class StormClientPipelineFactory implements ChannelPipelineFactory { private Client client; StormClientPipelineFactory(Client client) { this.client = client; } public ChannelPipeline getPipeline() throws Exception { // Create a default pipeline implementation. ChannelPipeline pipeline = Channels.pipeline(); // Decoder pipeline.addLast("decoder", new MessageDecoder()); // Encoder pipeline.addLast("encoder", new MessageEncoder()); // business logic. pipeline.addLast("handler", new StormClientHandler(client)); return pipeline; } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import java.util.concurrent.atomic.AtomicInteger; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.channel.ChannelStateEvent; import org.jboss.netty.channel.ExceptionEvent; import org.jboss.netty.channel.MessageEvent; import org.jboss.netty.channel.SimpleChannelUpstreamHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.messaging.TaskMessage; class StormServerHandler extends SimpleChannelUpstreamHandler { private static final Logger LOG = LoggerFactory.getLogger(StormServerHandler.class); Server server; private AtomicInteger failure_count; StormServerHandler(Server server) { this.server = server; failure_count = new AtomicInteger(0); } @Override public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) { server.addChannel(e.getChannel()); } @Override public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) { Object msg = e.getMessage(); if (msg == null) return; //end of batch? if (msg==ControlMessage.EOB_MESSAGE) { Channel channel = ctx.getChannel(); LOG.debug("Send back response ..."); if (failure_count.get()==0) channel.write(ControlMessage.OK_RESPONSE); else channel.write(ControlMessage.FAILURE_RESPONSE); return; } //enqueue the received message for processing try { server.enqueue((TaskMessage)msg); } catch (InterruptedException e1) { LOG.info("failed to enqueue a request message", e); failure_count.incrementAndGet(); } } @Override public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { server.closeChannel(e.getChannel()); } } ================================================ FILE: storm-netty/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java ================================================ /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package backtype.storm.messaging.netty; import org.jboss.netty.channel.ChannelPipeline; import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.Channels; class StormServerPipelineFactory implements ChannelPipelineFactory { private Server server; StormServerPipelineFactory(Server server) { this.server = server; } public ChannelPipeline getPipeline() throws Exception { // Create a default pipeline implementation. ChannelPipeline pipeline = Channels.pipeline(); // Decoder pipeline.addLast("decoder", new MessageDecoder()); // Encoder pipeline.addLast("encoder", new MessageEncoder()); // business logic. pipeline.addLast("handler", new StormServerHandler(server)); return pipeline; } } ================================================ FILE: storm-netty/test/clj/backtype/storm/messaging/netty_integration_test.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.messaging.netty-integration-test (:use [clojure test]) (:import [backtype.storm.messaging TransportFactory]) (:import [backtype.storm.testing TestWordSpout TestGlobalCount]) (:use [backtype.storm bootstrap testing util])) (bootstrap) (deftest test-integration (with-simulated-time-local-cluster [cluster :supervisors 4 :supervisor-slot-port-min 6710 :daemon-conf {STORM-LOCAL-MODE-ZMQ true STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 }] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)} {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.) :parallelism-hint 6)}) results (complete-topology cluster topology ;; important for test that ;; #tuples = multiple of 4 and 6 :storm-conf {TOPOLOGY-WORKERS 3} :mock-sources {"1" [["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ["a"] ["b"] ]} )] (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]])) (read-tuples results "2")))))) ================================================ FILE: storm-netty/test/clj/backtype/storm/messaging/netty_unit_test.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.messaging.netty-unit-test (:use [clojure test]) (:import [backtype.storm.messaging TransportFactory]) (:use [backtype.storm bootstrap testing util])) (bootstrap) (def port 6700) (def task 1) (deftest test-basic (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 } context (TransportFactory/makeContext storm-conf) server (.bind context nil port) client (.connect context nil "localhost" port) _ (.send client task (.getBytes req_msg)) resp (.recv server 0)] (is (= task (.task resp))) (is (= req_msg (String. (.message resp)))) (.close client) (.close server) (.term context))) (deftest test-large-msg (let [req_msg (apply str (repeat 2048000 'c')) storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 102400 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 } context (TransportFactory/makeContext storm-conf) server (.bind context nil port) client (.connect context nil "localhost" port) _ (.send client task (.getBytes req_msg)) resp (.recv server 0)] (is (= task (.task resp))) (is (= req_msg (String. (.message resp)))) (.close client) (.close server) (.term context))) (deftest test-server-delayed (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 } context (TransportFactory/makeContext storm-conf) client (.connect context nil "localhost" port) _ (.send client task (.getBytes req_msg)) _ (Thread/sleep 1000) server (.bind context nil port) resp (.recv server 0)] (is (= task (.task resp))) (is (= req_msg (String. (.message resp)))) (.close client) (.close server) (.term context))) (deftest test-batch (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 } context (TransportFactory/makeContext storm-conf) server (.bind context nil port) client (.connect context nil "localhost" port)] (doseq [num (range 1 100000)] (let [req_msg (str num)] (.send client task (.getBytes req_msg)))) (doseq [num (range 1 100000)] (let [req_msg (str num) resp (.recv server 0) resp_msg (String. (.message resp))] (is (= req_msg resp_msg)))) (.close client) (.close server) (.term context)))