├── README.md └── apache-storm-0.9.4 ├── .gitignore ├── BYLAWS.md ├── CHANGELOG.md ├── DEVELOPER.md ├── DISCLAIMER ├── KEYS ├── LICENSE ├── NOTICE ├── README.markdown ├── SECURITY.md ├── STORM-UI-REST-API.md ├── TODO ├── VERSION ├── bin ├── storm ├── storm-config.cmd └── storm.cmd ├── conf ├── defaults.yaml ├── jaas_digest.conf ├── logback.xml ├── storm.yaml.example └── storm_env.ini ├── dev-tools ├── github │ └── __init__.py ├── jira-github-join.py └── jira │ └── __init__.py ├── examples └── storm-starter │ ├── README.markdown │ ├── multilang │ └── resources │ │ ├── asyncSplitsentence.js │ │ ├── randomsentence.js │ │ ├── splitsentence.js │ │ ├── splitsentence.py │ │ ├── splitsentence.rb │ │ ├── storm.js │ │ ├── storm.py │ │ └── storm.rb │ ├── pom.xml │ ├── src │ ├── clj │ │ └── storm │ │ │ └── starter │ │ │ └── clj │ │ │ └── word_count.clj │ └── jvm │ │ └── storm │ │ └── starter │ │ ├── BasicDRPCTopology.java │ │ ├── ExclamationTopology.java │ │ ├── ManualDRPC.java │ │ ├── PrintSampleStream.java │ │ ├── ReachTopology.java │ │ ├── RollingTopWords.java │ │ ├── SingleJoinExample.java │ │ ├── TransactionalGlobalCount.java │ │ ├── TransactionalWords.java │ │ ├── WordCountTopology.java │ │ ├── WordCountTopologyNode.java │ │ ├── bolt │ │ ├── AbstractRankerBolt.java │ │ ├── IntermediateRankingsBolt.java │ │ ├── PrinterBolt.java │ │ ├── RollingCountBolt.java │ │ ├── SingleJoinBolt.java │ │ └── TotalRankingsBolt.java │ │ ├── spout │ │ ├── RandomSentenceSpout.java │ │ └── TwitterSampleSpout.java │ │ ├── tools │ │ ├── NthLastModifiedTimeTracker.java │ │ ├── Rankable.java │ │ ├── RankableObjectWithFields.java │ │ ├── Rankings.java │ │ ├── SlidingWindowCounter.java │ │ └── SlotBasedCounter.java │ │ ├── trident │ │ ├── TridentReach.java │ │ └── TridentWordCount.java │ │ └── util │ │ ├── StormRunner.java │ │ └── TupleHelpers.java │ └── test │ └── jvm │ └── storm │ └── starter │ ├── bolt │ ├── IntermediateRankingsBoltTest.java │ ├── RollingCountBoltTest.java │ └── TotalRankingsBoltTest.java │ └── tools │ ├── MockTupleHelpers.java │ ├── NthLastModifiedTimeTrackerTest.java │ ├── RankableObjectWithFieldsTest.java │ ├── RankingsTest.java │ ├── SlidingWindowCounterTest.java │ └── SlotBasedCounterTest.java ├── external ├── storm-hbase │ ├── LICENSE │ ├── README.md │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── storm │ │ │ └── hbase │ │ │ ├── bolt │ │ │ ├── AbstractHBaseBolt.java │ │ │ ├── HBaseBolt.java │ │ │ ├── HBaseLookupBolt.java │ │ │ └── mapper │ │ │ │ ├── HBaseMapper.java │ │ │ │ ├── HBaseProjectionCriteria.java │ │ │ │ ├── HBaseValueMapper.java │ │ │ │ └── SimpleHBaseMapper.java │ │ │ ├── common │ │ │ ├── ColumnList.java │ │ │ ├── HBaseClient.java │ │ │ ├── IColumn.java │ │ │ ├── ICounter.java │ │ │ └── Utils.java │ │ │ ├── security │ │ │ └── HBaseSecurityUtil.java │ │ │ └── trident │ │ │ ├── mapper │ │ │ ├── SimpleTridentHBaseMapper.java │ │ │ └── TridentHBaseMapper.java │ │ │ └── state │ │ │ ├── HBaseMapState.java │ │ │ ├── HBaseQuery.java │ │ │ ├── HBaseState.java │ │ │ ├── HBaseStateFactory.java │ │ │ └── HBaseUpdater.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── storm │ │ └── hbase │ │ ├── topology │ │ ├── LookupWordCount.java │ │ ├── PersistentWordCount.java │ │ ├── TotalWordCounter.java │ │ ├── WordCountClient.java │ │ ├── WordCountValueMapper.java │ │ ├── WordCounter.java │ │ └── WordSpout.java │ │ └── trident │ │ ├── PrintFunction.java │ │ └── WordCountTrident.java ├── storm-hdfs │ ├── README.md │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── storm │ │ │ └── hdfs │ │ │ ├── bolt │ │ │ ├── AbstractHdfsBolt.java │ │ │ ├── HdfsBolt.java │ │ │ ├── SequenceFileBolt.java │ │ │ ├── format │ │ │ │ ├── DefaultFileNameFormat.java │ │ │ │ ├── DefaultSequenceFormat.java │ │ │ │ ├── DelimitedRecordFormat.java │ │ │ │ ├── FileNameFormat.java │ │ │ │ ├── RecordFormat.java │ │ │ │ └── SequenceFormat.java │ │ │ ├── rotation │ │ │ │ ├── FileRotationPolicy.java │ │ │ │ ├── FileSizeRotationPolicy.java │ │ │ │ ├── NoRotationPolicy.java │ │ │ │ └── TimedRotationPolicy.java │ │ │ └── sync │ │ │ │ ├── CountSyncPolicy.java │ │ │ │ └── SyncPolicy.java │ │ │ ├── common │ │ │ ├── rotation │ │ │ │ ├── MoveFileAction.java │ │ │ │ └── RotationAction.java │ │ │ └── security │ │ │ │ └── HdfsSecurityUtil.java │ │ │ └── trident │ │ │ ├── HdfsState.java │ │ │ ├── HdfsStateFactory.java │ │ │ ├── HdfsUpdater.java │ │ │ ├── format │ │ │ ├── DefaultFileNameFormat.java │ │ │ ├── DefaultSequenceFormat.java │ │ │ ├── DelimitedRecordFormat.java │ │ │ ├── FileNameFormat.java │ │ │ ├── RecordFormat.java │ │ │ └── SequenceFormat.java │ │ │ ├── rotation │ │ │ ├── FileRotationPolicy.java │ │ │ ├── FileSizeRotationPolicy.java │ │ │ ├── NoRotationPolicy.java │ │ │ └── TimedRotationPolicy.java │ │ │ └── sync │ │ │ ├── CountSyncPolicy.java │ │ │ └── SyncPolicy.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── storm │ │ └── hdfs │ │ ├── bolt │ │ ├── HdfsFileTopology.java │ │ └── SequenceFileTopology.java │ │ └── trident │ │ ├── FixedBatchSpout.java │ │ ├── TridentFileTopology.java │ │ └── TridentSequenceTopology.java └── storm-kafka │ ├── CHANGELOG.md │ ├── README.md │ ├── pom.xml │ └── src │ ├── jvm │ └── storm │ │ └── kafka │ │ ├── Broker.java │ │ ├── BrokerHosts.java │ │ ├── DynamicBrokersReader.java │ │ ├── DynamicPartitionConnections.java │ │ ├── FailedFetchException.java │ │ ├── KafkaConfig.java │ │ ├── KafkaError.java │ │ ├── KafkaSpout.java │ │ ├── KafkaUtils.java │ │ ├── KeyValueScheme.java │ │ ├── KeyValueSchemeAsMultiScheme.java │ │ ├── Partition.java │ │ ├── PartitionCoordinator.java │ │ ├── PartitionManager.java │ │ ├── SpoutConfig.java │ │ ├── StaticCoordinator.java │ │ ├── StaticHosts.java │ │ ├── StaticPartitionConnections.java │ │ ├── StringKeyValueScheme.java │ │ ├── StringScheme.java │ │ ├── UpdateOffsetException.java │ │ ├── ZkCoordinator.java │ │ ├── ZkHosts.java │ │ ├── ZkState.java │ │ ├── bolt │ │ ├── KafkaBolt.java │ │ ├── mapper │ │ │ ├── FieldNameBasedTupleToKafkaMapper.java │ │ │ └── TupleToKafkaMapper.java │ │ └── selector │ │ │ ├── DefaultTopicSelector.java │ │ │ └── KafkaTopicSelector.java │ │ └── trident │ │ ├── Coordinator.java │ │ ├── DefaultCoordinator.java │ │ ├── GlobalPartitionInformation.java │ │ ├── IBatchCoordinator.java │ │ ├── IBrokerReader.java │ │ ├── MaxMetric.java │ │ ├── OpaqueTridentKafkaSpout.java │ │ ├── StaticBrokerReader.java │ │ ├── TransactionalTridentKafkaSpout.java │ │ ├── TridentKafkaConfig.java │ │ ├── TridentKafkaEmitter.java │ │ ├── TridentKafkaState.java │ │ ├── TridentKafkaStateFactory.java │ │ ├── TridentKafkaUpdater.java │ │ ├── ZkBrokerReader.java │ │ ├── mapper │ │ ├── FieldNameBasedTupleToKafkaMapper.java │ │ └── TridentTupleToKafkaMapper.java │ │ └── selector │ │ ├── DefaultTopicSelector.java │ │ └── KafkaTopicSelector.java │ └── test │ └── storm │ └── kafka │ ├── DynamicBrokersReaderTest.java │ ├── KafkaErrorTest.java │ ├── KafkaTestBroker.java │ ├── KafkaUtilsTest.java │ ├── StringKeyValueSchemeTest.java │ ├── TestUtils.java │ ├── TridentKafkaTest.java │ ├── TridentKafkaTopology.java │ ├── ZkCoordinatorTest.java │ └── bolt │ └── KafkaBoltTest.java ├── logback └── cluster.xml ├── pom.xml ├── storm-buildtools └── maven-shade-clojure-transformer │ ├── pom.xml │ └── src │ └── main │ └── java │ └── org │ └── apache │ └── storm │ └── maven │ └── shade │ └── clojure │ └── ClojureTransformer.java ├── storm-core ├── dependency-reduced-pom.xml ├── pom.xml ├── 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 │ │ │ │ ├── monitor.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 │ │ │ │ ├── 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 │ ├── dev │ │ └── resources │ │ │ ├── storm.js │ │ │ ├── storm.py │ │ │ ├── storm.rb │ │ │ ├── tester_bolt.js │ │ │ ├── tester_bolt.py │ │ │ ├── tester_bolt.rb │ │ │ ├── tester_bolt_metrics.py │ │ │ ├── tester_spout.js │ │ │ ├── tester_spout.py │ │ │ ├── tester_spout.rb │ │ │ └── tester_spout_metrics.py │ ├── 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 │ │ │ │ ├── ConnectionWithStatus.java │ │ │ │ ├── IConnection.java │ │ │ │ ├── IContext.java │ │ │ │ ├── TaskMessage.java │ │ │ │ ├── TransportFactory.java │ │ │ │ └── netty │ │ │ │ │ ├── Client.java │ │ │ │ │ ├── Context.java │ │ │ │ │ ├── ControlMessage.java │ │ │ │ │ ├── MessageBatch.java │ │ │ │ │ ├── MessageDecoder.java │ │ │ │ │ ├── MessageEncoder.java │ │ │ │ │ ├── NettyRenameThreadFactory.java │ │ │ │ │ ├── Server.java │ │ │ │ │ ├── StormClientErrorHandler.java │ │ │ │ │ ├── StormClientPipelineFactory.java │ │ │ │ │ ├── StormServerHandler.java │ │ │ │ │ └── StormServerPipelineFactory.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 │ │ │ │ │ └── rpc │ │ │ │ │ ├── AssignableShellMetric.java │ │ │ │ │ ├── CombinedShellMetric.java │ │ │ │ │ ├── CountShellMetric.java │ │ │ │ │ ├── IShellMetric.java │ │ │ │ │ └── ReducedShellMetric.java │ │ │ │ ├── multilang │ │ │ │ ├── BoltMsg.java │ │ │ │ ├── ISerializer.java │ │ │ │ ├── JsonSerializer.java │ │ │ │ ├── NoOutputException.java │ │ │ │ ├── ShellMsg.java │ │ │ │ └── SpoutMsg.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 │ │ │ │ ├── DefaultSerializationDelegate.java │ │ │ │ ├── GzipBridgeSerializationDelegate.java │ │ │ │ ├── GzipSerializationDelegate.java │ │ │ │ ├── IKryoDecorator.java │ │ │ │ ├── IKryoFactory.java │ │ │ │ ├── ITupleDeserializer.java │ │ │ │ ├── ITupleSerializer.java │ │ │ │ ├── KryoTupleDeserializer.java │ │ │ │ ├── KryoTupleSerializer.java │ │ │ │ ├── KryoValuesDeserializer.java │ │ │ │ ├── KryoValuesSerializer.java │ │ │ │ ├── SerializableSerializer.java │ │ │ │ ├── SerializationDelegate.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 │ │ │ │ ├── PythonShellMetricsBolt.java │ │ │ │ ├── PythonShellMetricsSpout.java │ │ │ │ ├── SpoutTracker.java │ │ │ │ ├── TestAggregatesCounter.java │ │ │ │ ├── TestConfBolt.java │ │ │ │ ├── TestEventLogSpout.java │ │ │ │ ├── TestEventOrderCheckBolt.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 │ │ │ │ ├── ITuple.java │ │ │ │ ├── MessageId.java │ │ │ │ ├── Tuple.java │ │ │ │ ├── TupleImpl.java │ │ │ │ └── Values.java │ │ │ │ ├── ui │ │ │ │ └── InvalidRequestException.java │ │ │ │ └── utils │ │ │ │ ├── BufferFileInputStream.java │ │ │ │ ├── CRC32OutputStream.java │ │ │ │ ├── ClojureTimerTask.java │ │ │ │ ├── Container.java │ │ │ │ ├── DRPCClient.java │ │ │ │ ├── DisruptorQueue.java │ │ │ │ ├── IndifferentAccessMap.java │ │ │ │ ├── InprocMessaging.java │ │ │ │ ├── KeyedRoundRobinQueue.java │ │ │ │ ├── ListDelegate.java │ │ │ │ ├── LocalState.java │ │ │ │ ├── Monitor.java │ │ │ │ ├── MutableInt.java │ │ │ │ ├── MutableLong.java │ │ │ │ ├── MutableObject.java │ │ │ │ ├── NimbusClient.java │ │ │ │ ├── RegisteredGlobalState.java │ │ │ │ ├── RotatingMap.java │ │ │ │ ├── ServiceRegistry.java │ │ │ │ ├── ShellProcess.java │ │ │ │ ├── StormBoundedExponentialBackoffRetry.java │ │ │ │ ├── ThriftTopologyUtils.java │ │ │ │ ├── Time.java │ │ │ │ ├── TimeCacheMap.java │ │ │ │ ├── TransferDrainer.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 │ │ │ │ ├── RemovableMapState.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 │ │ │ ├── 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 │ │ ├── js │ │ │ └── storm.js │ │ ├── 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 │ │ ├── component.html │ │ ├── css │ │ ├── bootstrap-1.4.0.css │ │ └── style.css │ │ ├── images │ │ └── spinner.gif │ │ ├── index.html │ │ ├── js │ │ ├── arbor-graphics.js │ │ ├── arbor-tween.js │ │ ├── arbor.js │ │ ├── bootstrap-twipsy.js │ │ ├── jquery-1.6.2.min.js │ │ ├── jquery.blockUI.min.js │ │ ├── jquery.cookies.2.2.0.min.js │ │ ├── jquery.mustache.js │ │ ├── jquery.tablesorter.min.js │ │ ├── moment.min.js │ │ ├── script.js │ │ ├── url.min.js │ │ └── visualization.js │ │ ├── templates │ │ ├── component-page-template.html │ │ ├── index-page-template.html │ │ ├── json-error-template.html │ │ └── topology-page-template.html │ │ └── topology.html └── 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 │ │ │ ├── netty_integration_test.clj │ │ │ └── netty_unit_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 │ │ │ └── worker_test.clj │ └── storm │ │ └── trident │ │ ├── integration_test.clj │ │ ├── state_test.clj │ │ └── tuple_test.clj │ ├── jvm │ └── backtype │ │ └── storm │ │ ├── serialization │ │ └── GzipBridgeSerializationDelegateTest.java │ │ └── utils │ │ ├── DisruptorQueueTest.java │ │ └── StormBoundedExponentialBackoffRetryTest.java │ └── multilang │ └── fy │ ├── bolt.fy │ ├── mocks.fy │ └── protocol.fy └── storm-dist ├── binary ├── LICENSE ├── NOTICE ├── pom.xml └── src │ └── main │ └── assembly │ └── binary.xml └── source ├── pom.xml └── src └── main └── assembly └── source.xml /README.md: -------------------------------------------------------------------------------- 1 | # storm-src 2 | i changer the storm src and allow it easy to use在web ui界面可以多显示一组数据 3 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/.gitignore: -------------------------------------------------------------------------------- 1 | .lein-repl-history 2 | /classes 3 | /lib 4 | deploy/lib 5 | deploy/logs 6 | .emacs-project 7 | *.jar 8 | bin/jzmq 9 | .DS_Store 10 | deploy/classes 11 | *.fyc 12 | *.rbc 13 | *.pyc 14 | CHILD 15 | CHILDMAKER 16 | NANNY 17 | \#project.clj\# 18 | .\#project.clj 19 | .lein-failures 20 | _release 21 | *.zip 22 | *.tar.gz 23 | .lein-deps-sum 24 | *.iml 25 | target 26 | /.project/ 27 | /.lein-plugins/ 28 | *.ipr 29 | *.iws 30 | .idea 31 | .* 32 | !/.gitignore 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/DISCLAIMER: -------------------------------------------------------------------------------- 1 | Apache Storm is an effort undergoing incubation at the Apache Software 2 | Foundation (ASF), sponsored by the Apache Incubator PMC. 3 | 4 | Incubation is required of all newly accepted projects until a further review 5 | indicates that the infrastructure, communications, and decision making process 6 | have stabilized in a manner consistent with other successful ASF projects. 7 | 8 | While incubation status is not necessarily a reflection of the completeness 9 | or stability of the code, it does indicate that the project has yet to be 10 | fully endorsed by the ASF. 11 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Storm 2 | Copyright 2014 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | 7 | This product includes software developed by Yahoo! Inc. (www.yahoo.com) 8 | Copyright (c) 2012-2014 Yahoo! Inc. -------------------------------------------------------------------------------- /apache-storm-0.9.4/VERSION: -------------------------------------------------------------------------------- 1 | ${project.version} 2 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/conf/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 20 | 21 | %-4r [%t] %-5p %c - %m%n 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/conf/storm_env.ini: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | # 18 | 19 | 20 | # Environment variables in the following section will be used 21 | # in storm python script. They override the environment variables 22 | # set in the shell. 23 | [environment] 24 | 25 | # The java implementation to use. If JAVA_HOME is not found we expect java to be in path 26 | #JAVA_HOME:home 27 | 28 | # JVM options to be used in "storm jar" commad 29 | #STORM_JAR_JVM_OPTS: 30 | 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/examples/storm-starter/multilang/resources/splitsentence.py: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | import storm 17 | 18 | class SplitSentenceBolt(storm.BasicBolt): 19 | def process(self, tup): 20 | words = tup.values[0].split(" ") 21 | for word in words: 22 | storm.emit([word]) 23 | 24 | SplitSentenceBolt().run() -------------------------------------------------------------------------------- /apache-storm-0.9.4/examples/storm-starter/multilang/resources/splitsentence.rb: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | require "./storm" 17 | 18 | class SplitSentenceBolt < Storm::Bolt 19 | def process(tup) 20 | tup.values[0].split(" ").each do |word| 21 | emit([word]) 22 | end 23 | end 24 | end 25 | 26 | SplitSentenceBolt.new.run -------------------------------------------------------------------------------- /apache-storm-0.9.4/examples/storm-starter/src/jvm/storm/starter/tools/Rankable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.starter.tools; 19 | 20 | public interface Rankable extends Comparable { 21 | 22 | Object getObject(); 23 | 24 | long getCount(); 25 | 26 | /** 27 | * Note: We do not defensively copy the object wrapped by the Rankable. It is passed as is. 28 | * 29 | * @return a defensive copy 30 | */ 31 | Rankable copy(); 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/examples/storm-starter/src/jvm/storm/starter/util/TupleHelpers.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.starter.util; 19 | 20 | import backtype.storm.Constants; 21 | import backtype.storm.tuple.Tuple; 22 | 23 | public final class TupleHelpers { 24 | 25 | private TupleHelpers() { 26 | } 27 | 28 | public static boolean isTickTuple(Tuple tuple) { 29 | return tuple.getSourceComponent().equals(Constants.SYSTEM_COMPONENT_ID) && tuple.getSourceStreamId().equals( 30 | Constants.SYSTEM_TICK_STREAM_ID); 31 | } 32 | 33 | } 34 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-hbase/src/main/java/org/apache/storm/hbase/common/IColumn.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.storm.hbase.common; 19 | 20 | /** 21 | * Interface definition for classes that support being written to HBase as 22 | * a regular column. 23 | * 24 | */ 25 | public interface IColumn { 26 | byte[] family(); 27 | byte[] qualifier(); 28 | byte[] value(); 29 | long timestamp(); 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-hbase/src/main/java/org/apache/storm/hbase/common/ICounter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.storm.hbase.common; 19 | 20 | /** 21 | * Interface definition for classes that support being written to HBase as 22 | * a counter column. 23 | * 24 | */ 25 | public interface ICounter { 26 | byte[] family(); 27 | byte[] qualifier(); 28 | long increment(); 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/format/RecordFormat.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.storm.hdfs.bolt.format; 19 | 20 | 21 | import backtype.storm.tuple.Tuple; 22 | 23 | import java.io.Serializable; 24 | 25 | /** 26 | * Formats a Tuple object into a byte array 27 | * that will be written to HDFS. 28 | * 29 | */ 30 | public interface RecordFormat extends Serializable { 31 | byte[] format(Tuple tuple); 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/rotation/NoRotationPolicy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.storm.hdfs.bolt.rotation; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | /** 23 | * File rotation policy that will never rotate... 24 | * Just one big file. Intended for testing purposes. 25 | */ 26 | public class NoRotationPolicy implements FileRotationPolicy { 27 | @Override 28 | public boolean mark(Tuple tuple, long offset) { 29 | return false; 30 | } 31 | 32 | @Override 33 | public void reset() { 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/rotation/RotationAction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.storm.hdfs.common.rotation; 19 | 20 | 21 | import org.apache.hadoop.fs.FileSystem; 22 | import org.apache.hadoop.fs.Path; 23 | 24 | import java.io.IOException; 25 | import java.io.Serializable; 26 | 27 | public interface RotationAction extends Serializable { 28 | void execute(FileSystem fileSystem, Path filePath) throws IOException; 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/trident/HdfsUpdater.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.storm.hdfs.trident; 19 | 20 | import storm.trident.operation.TridentCollector; 21 | import storm.trident.state.BaseStateUpdater; 22 | import storm.trident.tuple.TridentTuple; 23 | 24 | import java.util.List; 25 | 26 | public class HdfsUpdater extends BaseStateUpdater{ 27 | @Override 28 | public void updateState(HdfsState state, List tuples, TridentCollector collector) { 29 | state.updateState(tuples, collector); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/trident/format/RecordFormat.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.storm.hdfs.trident.format; 19 | 20 | 21 | import storm.trident.tuple.TridentTuple; 22 | 23 | import java.io.Serializable; 24 | 25 | /** 26 | * Formats a Tuple object into a byte array 27 | * that will be written to HDFS. 28 | * 29 | */ 30 | public interface RecordFormat extends Serializable { 31 | byte[] format(TridentTuple tuple); 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/CHANGELOG.md: -------------------------------------------------------------------------------- 1 | ## 0.9.2-incubating (0.5.0) 2 | * incorporated as an Apache Storm external module 3 | * fixed partition assignment for KafkaSpout 4 | * upgraded to storm 0.9.1 5 | 6 | ## 0.4.0 7 | * added support for reading kafka message keys 8 | * configurable metrics emit interval 9 | 10 | ## 0.3.0 11 | * updated partition path in zookeeper 12 | * added error handling for fetch request 13 | 14 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/BrokerHosts.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka; 19 | 20 | import java.io.Serializable; 21 | 22 | 23 | public interface BrokerHosts extends Serializable { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/FailedFetchException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka; 19 | 20 | public class FailedFetchException extends RuntimeException { 21 | 22 | public FailedFetchException(String message) { 23 | super(message); 24 | } 25 | 26 | public FailedFetchException(Exception e) { 27 | super(e); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka; 19 | 20 | import backtype.storm.spout.Scheme; 21 | 22 | import java.util.List; 23 | 24 | public interface KeyValueScheme extends Scheme { 25 | 26 | public List deserializeKeyAndValue(byte[] key, byte[] value); 27 | 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/PartitionCoordinator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka; 19 | 20 | import java.util.List; 21 | 22 | public interface PartitionCoordinator { 23 | List getMyManagedPartitions(); 24 | 25 | PartitionManager getManager(Partition partition); 26 | 27 | void refresh(); 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/UpdateOffsetException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka; 19 | 20 | public class UpdateOffsetException extends RuntimeException { 21 | 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/bolt/mapper/TupleToKafkaMapper.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.bolt.mapper; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | import java.io.Serializable; 23 | 24 | /** 25 | * as the really verbose name suggests this interface mapps a storm tuple to kafka key and message. 26 | * @param type of key. 27 | * @param type of value. 28 | */ 29 | public interface TupleToKafkaMapper extends Serializable { 30 | K getKeyFromTuple(Tuple tuple); 31 | V getMessageFromTuple(Tuple tuple); 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/bolt/selector/DefaultTopicSelector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.bolt.selector; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | public class DefaultTopicSelector implements KafkaTopicSelector { 23 | 24 | private final String topicName; 25 | 26 | public DefaultTopicSelector(final String topicName) { 27 | this.topicName = topicName; 28 | } 29 | 30 | @Override 31 | public String getTopic(Tuple tuple) { 32 | return topicName; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/bolt/selector/KafkaTopicSelector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.bolt.selector; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | import java.io.Serializable; 23 | 24 | public interface KafkaTopicSelector extends Serializable { 25 | String getTopic(Tuple tuple); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/trident/DefaultCoordinator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.trident; 19 | 20 | public class DefaultCoordinator implements IBatchCoordinator { 21 | 22 | @Override 23 | public boolean isReady(long txid) { 24 | return true; 25 | } 26 | 27 | @Override 28 | public void close() { 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/trident/IBatchCoordinator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.trident; 19 | 20 | import java.io.Serializable; 21 | 22 | public interface IBatchCoordinator extends Serializable { 23 | boolean isReady(long txid); 24 | 25 | void close(); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.trident; 19 | 20 | public interface IBrokerReader { 21 | 22 | GlobalPartitionInformation getCurrentBrokers(); 23 | 24 | void close(); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaUpdater.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.trident; 19 | 20 | import storm.trident.operation.TridentCollector; 21 | import storm.trident.state.BaseStateUpdater; 22 | import storm.trident.tuple.TridentTuple; 23 | 24 | import java.util.List; 25 | 26 | public class TridentKafkaUpdater extends BaseStateUpdater { 27 | @Override 28 | public void updateState(TridentKafkaState state, List tuples, TridentCollector collector) { 29 | state.updateState(tuples, collector); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.trident.mapper; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | import storm.trident.tuple.TridentTuple; 22 | 23 | import java.io.Serializable; 24 | 25 | public interface TridentTupleToKafkaMapper extends Serializable { 26 | K getKeyFromTuple(TridentTuple tuple); 27 | V getMessageFromTuple(TridentTuple tuple); 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/trident/selector/DefaultTopicSelector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.trident.selector; 19 | 20 | import storm.trident.tuple.TridentTuple; 21 | 22 | public class DefaultTopicSelector implements KafkaTopicSelector { 23 | 24 | private final String topicName; 25 | 26 | public DefaultTopicSelector(final String topicName) { 27 | this.topicName = topicName; 28 | } 29 | 30 | @Override 31 | public String getTopic(TridentTuple tuple) { 32 | return topicName; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/external/storm-kafka/src/jvm/storm/kafka/trident/selector/KafkaTopicSelector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.kafka.trident.selector; 19 | 20 | import storm.trident.tuple.TridentTuple; 21 | 22 | import java.io.Serializable; 23 | 24 | public interface KafkaTopicSelector extends Serializable { 25 | String getTopic(TridentTuple tuple); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/clj/backtype/storm/command/activate.clj: -------------------------------------------------------------------------------- 1 | ;; Licensed to the Apache Software Foundation (ASF) under one 2 | ;; or more contributor license agreements. See the NOTICE file 3 | ;; distributed with this work for additional information 4 | ;; regarding copyright ownership. The ASF licenses this file 5 | ;; to you under the Apache License, Version 2.0 (the 6 | ;; "License"); you may not use this file except in compliance 7 | ;; with the License. You may obtain a copy of the License at 8 | ;; 9 | ;; http://www.apache.org/licenses/LICENSE-2.0 10 | ;; 11 | ;; Unless required by applicable law or agreed to in writing, software 12 | ;; distributed under the License is distributed on an "AS IS" BASIS, 13 | ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | ;; See the License for the specific language governing permissions and 15 | ;; limitations under the License. 16 | (ns backtype.storm.command.activate 17 | (:use [backtype.storm thrift log]) 18 | (:gen-class)) 19 | 20 | (defn -main [name] 21 | (with-configured-nimbus-connection nimbus 22 | (.activate nimbus name) 23 | (log-message "Activated topology: " name) 24 | )) 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/clj/backtype/storm/command/config_value.clj: -------------------------------------------------------------------------------- 1 | ;; Licensed to the Apache Software Foundation (ASF) under one 2 | ;; or more contributor license agreements. See the NOTICE file 3 | ;; distributed with this work for additional information 4 | ;; regarding copyright ownership. The ASF licenses this file 5 | ;; to you under the Apache License, Version 2.0 (the 6 | ;; "License"); you may not use this file except in compliance 7 | ;; with the License. You may obtain a copy of the License at 8 | ;; 9 | ;; http://www.apache.org/licenses/LICENSE-2.0 10 | ;; 11 | ;; Unless required by applicable law or agreed to in writing, software 12 | ;; distributed under the License is distributed on an "AS IS" BASIS, 13 | ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | ;; See the License for the specific language governing permissions and 15 | ;; limitations under the License. 16 | (ns backtype.storm.command.config-value 17 | (:use [backtype.storm config log]) 18 | (:gen-class)) 19 | 20 | 21 | (defn -main [^String name] 22 | (let [conf (read-storm-config)] 23 | (println "VALUE:" (conf name)) 24 | )) 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/clj/backtype/storm/command/deactivate.clj: -------------------------------------------------------------------------------- 1 | ;; Licensed to the Apache Software Foundation (ASF) under one 2 | ;; or more contributor license agreements. See the NOTICE file 3 | ;; distributed with this work for additional information 4 | ;; regarding copyright ownership. The ASF licenses this file 5 | ;; to you under the Apache License, Version 2.0 (the 6 | ;; "License"); you may not use this file except in compliance 7 | ;; with the License. You may obtain a copy of the License at 8 | ;; 9 | ;; http://www.apache.org/licenses/LICENSE-2.0 10 | ;; 11 | ;; Unless required by applicable law or agreed to in writing, software 12 | ;; distributed under the License is distributed on an "AS IS" BASIS, 13 | ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | ;; See the License for the specific language governing permissions and 15 | ;; limitations under the License. 16 | (ns backtype.storm.command.deactivate 17 | (:use [backtype.storm thrift log]) 18 | (:gen-class)) 19 | 20 | (defn -main [name] 21 | (with-configured-nimbus-connection nimbus 22 | (.deactivate nimbus name) 23 | (log-message "Deactivated topology: " name) 24 | )) 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/clj/backtype/storm/command/dev_zookeeper.clj: -------------------------------------------------------------------------------- 1 | ;; Licensed to the Apache Software Foundation (ASF) under one 2 | ;; or more contributor license agreements. See the NOTICE file 3 | ;; distributed with this work for additional information 4 | ;; regarding copyright ownership. The ASF licenses this file 5 | ;; to you under the Apache License, Version 2.0 (the 6 | ;; "License"); you may not use this file except in compliance 7 | ;; with the License. You may obtain a copy of the License at 8 | ;; 9 | ;; http://www.apache.org/licenses/LICENSE-2.0 10 | ;; 11 | ;; Unless required by applicable law or agreed to in writing, software 12 | ;; distributed under the License is distributed on an "AS IS" BASIS, 13 | ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | ;; See the License for the specific language governing permissions and 15 | ;; limitations under the License. 16 | (ns backtype.storm.command.dev-zookeeper 17 | (:use [backtype.storm zookeeper util config]) 18 | (:gen-class)) 19 | 20 | (defn -main [& args] 21 | (let [conf (read-storm-config) 22 | port (conf STORM-ZOOKEEPER-PORT) 23 | localpath (conf DEV-ZOOKEEPER-PATH)] 24 | (rmr localpath) 25 | (mk-inprocess-zookeeper localpath :port port) 26 | )) 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/clj/backtype/storm/tuple.clj: -------------------------------------------------------------------------------- 1 | ;; Licensed to the Apache Software Foundation (ASF) under one 2 | ;; or more contributor license agreements. See the NOTICE file 3 | ;; distributed with this work for additional information 4 | ;; regarding copyright ownership. The ASF licenses this file 5 | ;; to you under the Apache License, Version 2.0 (the 6 | ;; "License"); you may not use this file except in compliance 7 | ;; with the License. You may obtain a copy of the License at 8 | ;; 9 | ;; http://www.apache.org/licenses/LICENSE-2.0 10 | ;; 11 | ;; Unless required by applicable law or agreed to in writing, software 12 | ;; distributed under the License is distributed on an "AS IS" BASIS, 13 | ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | ;; See the License for the specific language governing permissions and 15 | ;; limitations under the License. 16 | 17 | (ns backtype.storm.tuple 18 | (:use [backtype.storm bootstrap])) 19 | 20 | (bootstrap) 21 | 22 | (defn list-hash-code 23 | [^List alist] 24 | (.hashCode alist)) 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/dev/resources/tester_bolt.rb: -------------------------------------------------------------------------------- 1 | # -*- coding: utf-8 -*- 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | 19 | require File.expand_path("storm", File.dirname(__FILE__)) 20 | 21 | class TesterBolt < Storm::Bolt 22 | def prepare(conf, context) 23 | emit ['bolt initializing'] 24 | end 25 | 26 | def process(tuple) 27 | word = tuple.values[0] 28 | if (rand < 0.75) 29 | emit [word + "lalala"], :anchor => tuple 30 | ack tuple 31 | else 32 | log(word + ' randomly skipped!') 33 | end 34 | end 35 | end 36 | 37 | TesterBolt.new.run 38 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/dev/resources/tester_bolt_metrics.py: -------------------------------------------------------------------------------- 1 | # -*- coding: utf-8 -*- 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | 19 | # This Python file uses the following encoding: utf-8 20 | 21 | import storm 22 | from random import random 23 | 24 | class TesterMetricsBolt(storm.Bolt): 25 | def initialize(self, conf, context): 26 | storm.log('bolt initializing') 27 | 28 | def process(self, tup): 29 | word = tup.values[0]; 30 | 31 | storm.rpcMetrics("my-custom-shell-metric", 1); 32 | 33 | storm.ack(tup) 34 | 35 | TesterMetricsBolt().run() 36 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/genthrift.sh: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | rm -rf gen-javabean gen-py py 18 | rm -rf jvm/backtype/storm/generated 19 | thrift7 --gen java:beans,hashcode,nocamel --gen py:utf8strings storm.thrift 20 | mv gen-javabean/backtype/storm/generated jvm/backtype/storm/generated 21 | mv gen-py py 22 | rm -rf gen-javabean 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/ILocalDRPC.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm; 19 | 20 | import backtype.storm.daemon.Shutdownable; 21 | import backtype.storm.generated.DistributedRPC; 22 | import backtype.storm.generated.DistributedRPCInvocations; 23 | 24 | 25 | public interface ILocalDRPC extends DistributedRPC.Iface, DistributedRPCInvocations.Iface, Shutdownable { 26 | public String getServiceId(); 27 | } 28 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/coordination/IBatchBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.coordination; 19 | 20 | import backtype.storm.task.TopologyContext; 21 | import backtype.storm.topology.IComponent; 22 | import backtype.storm.tuple.Tuple; 23 | import java.io.Serializable; 24 | import java.util.Map; 25 | 26 | public interface IBatchBolt extends Serializable, IComponent { 27 | void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, T id); 28 | void execute(Tuple tuple); 29 | void finishBatch(); 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/daemon/Shutdownable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.daemon; 19 | 20 | public interface Shutdownable { 21 | public void shutdown(); 22 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/hooks/info/BoltAckInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.hooks.info; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | public class BoltAckInfo { 23 | public Tuple tuple; 24 | public int ackingTaskId; 25 | public Long processLatencyMs; // null if it wasn't sampled 26 | 27 | public BoltAckInfo(Tuple tuple, int ackingTaskId, Long processLatencyMs) { 28 | this.tuple = tuple; 29 | this.ackingTaskId = ackingTaskId; 30 | this.processLatencyMs = processLatencyMs; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/hooks/info/BoltExecuteInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.hooks.info; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | public class BoltExecuteInfo { 23 | public Tuple tuple; 24 | public int executingTaskId; 25 | public Long executeLatencyMs; // null if it wasn't sampled 26 | 27 | public BoltExecuteInfo(Tuple tuple, int executingTaskId, Long executeLatencyMs) { 28 | this.tuple = tuple; 29 | this.executingTaskId = executingTaskId; 30 | this.executeLatencyMs = executeLatencyMs; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/hooks/info/BoltFailInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.hooks.info; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | public class BoltFailInfo { 23 | public Tuple tuple; 24 | public int failingTaskId; 25 | public Long failLatencyMs; // null if it wasn't sampled 26 | 27 | public BoltFailInfo(Tuple tuple, int failingTaskId, Long failLatencyMs) { 28 | this.tuple = tuple; 29 | this.failingTaskId = failingTaskId; 30 | this.failLatencyMs = failLatencyMs; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/hooks/info/SpoutAckInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.hooks.info; 19 | 20 | public class SpoutAckInfo { 21 | public Object messageId; 22 | public int spoutTaskId; 23 | public Long completeLatencyMs; // null if it wasn't sampled 24 | 25 | public SpoutAckInfo(Object messageId, int spoutTaskId, Long completeLatencyMs) { 26 | this.messageId = messageId; 27 | this.spoutTaskId = spoutTaskId; 28 | this.completeLatencyMs = completeLatencyMs; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/hooks/info/SpoutFailInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.hooks.info; 19 | 20 | public class SpoutFailInfo { 21 | public Object messageId; 22 | public int spoutTaskId; 23 | public Long failLatencyMs; // null if it wasn't sampled 24 | 25 | public SpoutFailInfo(Object messageId, int spoutTaskId, Long failLatencyMs) { 26 | this.messageId = messageId; 27 | this.spoutTaskId = spoutTaskId; 28 | this.failLatencyMs = failLatencyMs; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/AssignableMetric.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api; 19 | 20 | public class AssignableMetric implements IMetric { 21 | Object _value; 22 | 23 | public AssignableMetric(Object value) { 24 | _value = value; 25 | } 26 | 27 | public void setValue(Object value) { 28 | _value = value; 29 | } 30 | 31 | public Object getValueAndReset() { 32 | return _value; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/ICombiner.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api; 19 | 20 | public interface ICombiner { 21 | public T identity(); 22 | public T combine(T a, T b); 23 | } 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/IMetric.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api; 19 | 20 | public interface IMetric { 21 | public Object getValueAndReset(); 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/IReducer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api; 19 | 20 | public interface IReducer { 21 | T init(); 22 | T reduce(T accumulator, Object input); 23 | Object extractResult(T accumulator); 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/IStatefulObject.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api; 19 | 20 | public interface IStatefulObject { 21 | Object getState(); 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/StateMetric.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api; 19 | 20 | public class StateMetric implements IMetric { 21 | private IStatefulObject _obj; 22 | 23 | public StateMetric(IStatefulObject obj) { 24 | _obj = obj; 25 | } 26 | 27 | @Override 28 | public Object getValueAndReset() { 29 | return _obj.getState(); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/rpc/AssignableShellMetric.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api.rpc; 19 | 20 | import backtype.storm.metric.api.AssignableMetric; 21 | 22 | public class AssignableShellMetric extends AssignableMetric implements IShellMetric { 23 | public AssignableShellMetric(Object value) { 24 | super(value); 25 | } 26 | 27 | public void updateMetricFromRPC(Object value) { 28 | setValue(value); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/rpc/CombinedShellMetric.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api.rpc; 19 | 20 | import backtype.storm.metric.api.CombinedMetric; 21 | import backtype.storm.metric.api.ICombiner; 22 | 23 | public class CombinedShellMetric extends CombinedMetric implements IShellMetric { 24 | public CombinedShellMetric(ICombiner combiner) { 25 | super(combiner); 26 | } 27 | 28 | public void updateMetricFromRPC(Object value) { 29 | update(value); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/metric/api/rpc/ReducedShellMetric.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.metric.api.rpc; 19 | 20 | import backtype.storm.metric.api.IReducer; 21 | import backtype.storm.metric.api.ReducedMetric; 22 | 23 | public class ReducedShellMetric extends ReducedMetric implements IShellMetric { 24 | 25 | public ReducedShellMetric(IReducer reducer) { 26 | super(reducer); 27 | } 28 | 29 | public void updateMetricFromRPC(Object value) { 30 | update(value); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/nimbus/DefaultTopologyValidator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.nimbus; 19 | 20 | import backtype.storm.generated.InvalidTopologyException; 21 | import backtype.storm.generated.StormTopology; 22 | import java.util.Map; 23 | 24 | public class DefaultTopologyValidator implements ITopologyValidator { 25 | @Override 26 | public void prepare(Map StormConf){ 27 | } 28 | @Override 29 | public void validate(String topologyName, Map topologyConf, StormTopology topology) throws InvalidTopologyException { 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/nimbus/ITopologyValidator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.nimbus; 19 | 20 | import backtype.storm.generated.InvalidTopologyException; 21 | import backtype.storm.generated.StormTopology; 22 | import java.util.Map; 23 | 24 | public interface ITopologyValidator { 25 | void prepare(Map StormConf); 26 | void validate(String topologyName, Map topologyConf, StormTopology topology) 27 | throws InvalidTopologyException; 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/planner/CompoundSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.planner; 19 | 20 | 21 | public class CompoundSpout 22 | //implements ISpout 23 | { 24 | 25 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/planner/CompoundTask.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.planner; 19 | 20 | 21 | public class CompoundTask 22 | // implements IBolt 23 | { 24 | 25 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/planner/TaskBundle.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.planner; 19 | 20 | import backtype.storm.task.IBolt; 21 | import java.io.Serializable; 22 | 23 | 24 | public class TaskBundle implements Serializable { 25 | public IBolt task; 26 | public int componentId; 27 | 28 | public TaskBundle(IBolt task, int componentId) { 29 | this.task = task; 30 | this.componentId = componentId; 31 | } 32 | 33 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/serialization/IKryoDecorator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.serialization; 19 | import com.esotericsoftware.kryo.Kryo; 20 | 21 | public interface IKryoDecorator { 22 | void decorate(Kryo k); 23 | } 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/serialization/ITupleDeserializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.serialization; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | import java.io.IOException; 22 | 23 | public interface ITupleDeserializer { 24 | Tuple deserialize(byte[] ser); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/serialization/ITupleSerializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.serialization; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | 23 | public interface ITupleSerializer { 24 | byte[] serialize(Tuple tuple); 25 | // long crc32(Tuple tuple); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/serialization/SerializationDelegate.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.serialization; 19 | 20 | import java.util.Map; 21 | 22 | /** 23 | * Allow {@link backtype.storm.utils.Utils} to delegate meta serialization. 24 | */ 25 | public interface SerializationDelegate { 26 | 27 | /** 28 | * Lifecycle step that will be called after instantiating with nullary constructor. 29 | */ 30 | void prepare(Map stormConf); 31 | 32 | byte[] serialize(Object object); 33 | 34 | Object deserialize(byte[] bytes); 35 | } 36 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/serialization/types/HashMapSerializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.serialization.types; 19 | 20 | import com.esotericsoftware.kryo.Kryo; 21 | import com.esotericsoftware.kryo.io.Input; 22 | import com.esotericsoftware.kryo.serializers.MapSerializer; 23 | import java.util.HashMap; 24 | import java.util.Map; 25 | 26 | 27 | public class HashMapSerializer extends MapSerializer { 28 | @Override 29 | public Map create(Kryo kryo, Input input, Class type) { 30 | return new HashMap(); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/IMultiSchemableSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | public interface IMultiSchemableSpout { 21 | MultiScheme getScheme(); 22 | void setScheme(MultiScheme scheme); 23 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/ISchemableSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | 21 | public interface ISchemableSpout { 22 | Scheme getScheme(); 23 | void setScheme(Scheme scheme); 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/ISpoutOutputCollector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | import java.util.List; 21 | 22 | public interface ISpoutOutputCollector { 23 | /** 24 | Returns the task ids that received the tuples. 25 | */ 26 | List emit(String streamId, List tuple, Object messageId); 27 | void emitDirect(int taskId, String streamId, List tuple, Object messageId); 28 | void reportError(Throwable error); 29 | } 30 | 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/ISpoutWaitStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | import java.util.Map; 21 | 22 | /** 23 | * The strategy a spout needs to use when its waiting. Waiting is 24 | * triggered in one of two conditions: 25 | * 26 | * 1. nextTuple emits no tuples 27 | * 2. The spout has hit maxSpoutPending and can't emit any more tuples 28 | * 29 | * The default strategy sleeps for one millisecond. 30 | */ 31 | public interface ISpoutWaitStrategy { 32 | void prepare(Map conf); 33 | void emptyEmit(long streak); 34 | } 35 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/MultiScheme.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | import java.util.List; 21 | import java.io.Serializable; 22 | 23 | import backtype.storm.tuple.Fields; 24 | 25 | public interface MultiScheme extends Serializable { 26 | public Iterable> deserialize(byte[] ser); 27 | public Fields getOutputFields(); 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/NothingEmptyEmitStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | import java.util.Map; 21 | 22 | public class NothingEmptyEmitStrategy implements ISpoutWaitStrategy { 23 | @Override 24 | public void emptyEmit(long streak) { 25 | } 26 | 27 | @Override 28 | public void prepare(Map conf) { 29 | throw new UnsupportedOperationException("Not supported yet."); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/RawMultiScheme.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | import java.util.List; 21 | 22 | import backtype.storm.tuple.Fields; 23 | 24 | 25 | import static backtype.storm.utils.Utils.tuple; 26 | import static java.util.Arrays.asList; 27 | 28 | public class RawMultiScheme implements MultiScheme { 29 | @Override 30 | public Iterable> deserialize(byte[] ser) { 31 | return asList(tuple(ser)); 32 | } 33 | 34 | @Override 35 | public Fields getOutputFields() { 36 | return new Fields("bytes"); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/RawScheme.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | import backtype.storm.tuple.Fields; 21 | import java.util.List; 22 | import static backtype.storm.utils.Utils.tuple; 23 | 24 | public class RawScheme implements Scheme { 25 | public List deserialize(byte[] ser) { 26 | return tuple(ser); 27 | } 28 | 29 | public Fields getOutputFields() { 30 | return new Fields("bytes"); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/spout/Scheme.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.spout; 19 | 20 | import backtype.storm.tuple.Fields; 21 | import java.io.Serializable; 22 | import java.util.List; 23 | 24 | 25 | public interface Scheme extends Serializable { 26 | public List deserialize(byte[] ser); 27 | public Fields getOutputFields(); 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/state/IStateSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.state; 19 | 20 | import backtype.storm.task.TopologyContext; 21 | import java.io.Serializable; 22 | import java.util.Map; 23 | 24 | public interface IStateSpout extends Serializable { 25 | void open(Map conf, TopologyContext context); 26 | void close(); 27 | void nextTuple(StateSpoutOutputCollector collector); 28 | void synchronize(SynchronizeOutputCollector collector); 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/state/IStateSpoutOutputCollector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.state; 19 | 20 | public interface IStateSpoutOutputCollector extends ISynchronizeOutputCollector { 21 | void remove(int streamId, Object id); 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/state/ISubscribedState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.state; 19 | 20 | import backtype.storm.tuple.Tuple; 21 | 22 | public interface ISubscribedState { 23 | void set(Object id, Tuple tuple); 24 | void remove(Object id); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/state/ISynchronizeOutputCollector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.state; 19 | 20 | import java.util.List; 21 | 22 | public interface ISynchronizeOutputCollector { 23 | void add(int streamId, Object id, List tuple); 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/state/StateSpoutOutputCollector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.state; 19 | 20 | 21 | public class StateSpoutOutputCollector extends SynchronizeOutputCollector implements IStateSpoutOutputCollector { 22 | 23 | @Override 24 | public void remove(int streamId, Object id) { 25 | throw new UnsupportedOperationException("Not supported yet."); 26 | } 27 | 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/state/SynchronizeOutputCollector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.state; 19 | 20 | import java.util.List; 21 | 22 | 23 | public class SynchronizeOutputCollector implements ISynchronizeOutputCollector { 24 | 25 | @Override 26 | public void add(int streamId, Object id, List tuple) { 27 | throw new UnsupportedOperationException("Not supported yet."); 28 | } 29 | 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/task/IErrorReporter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.task; 19 | 20 | public interface IErrorReporter { 21 | void reportError(Throwable error); 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/testing/AckFailDelegate.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.testing; 19 | 20 | import java.io.Serializable; 21 | 22 | public interface AckFailDelegate extends Serializable { 23 | public void ack(Object id); 24 | public void fail(Object id); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/testing/KeyedCountingCommitterBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.testing; 19 | 20 | import backtype.storm.transactional.ICommitter; 21 | 22 | public class KeyedCountingCommitterBolt extends KeyedCountingBatchBolt implements ICommitter { 23 | 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/testing/TestKryoDecorator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.testing; 19 | 20 | import backtype.storm.serialization.IKryoDecorator; 21 | import com.esotericsoftware.kryo.Kryo; 22 | import com.esotericsoftware.kryo.Serializer; 23 | import com.esotericsoftware.kryo.io.Input; 24 | import com.esotericsoftware.kryo.io.Output; 25 | 26 | public class TestKryoDecorator implements IKryoDecorator { 27 | 28 | public void decorate(Kryo k) { 29 | k.register(TestSerObject.class); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/testing/TestSerObject.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.testing; 19 | 20 | import java.io.Serializable; 21 | 22 | public class TestSerObject implements Serializable { 23 | public int f1; 24 | public int f2; 25 | 26 | public TestSerObject(int f1, int f2) { 27 | this.f1 = f1; 28 | this.f2 = f2; 29 | } 30 | 31 | @Override 32 | public boolean equals(Object o) { 33 | TestSerObject other = (TestSerObject) o; 34 | return f1 == other.f1 && f2 == other.f2; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/testing/TrackedTopology.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.testing; 19 | 20 | import java.util.HashMap; 21 | import java.util.Map; 22 | 23 | import backtype.storm.generated.StormTopology; 24 | import clojure.lang.Keyword; 25 | 26 | public class TrackedTopology extends HashMap{ 27 | public TrackedTopology(Map map) { 28 | super(map); 29 | } 30 | 31 | public StormTopology getTopology() { 32 | return (StormTopology)get(Keyword.intern("topology")); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/BoltDeclarer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | public interface BoltDeclarer extends InputDeclarer, ComponentConfigurationDeclarer { 21 | 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/ComponentConfigurationDeclarer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | import java.util.Map; 21 | 22 | public interface ComponentConfigurationDeclarer { 23 | T addConfigurations(Map conf); 24 | T addConfiguration(String config, Object value); 25 | T setDebug(boolean debug); 26 | T setMaxTaskParallelism(Number val); 27 | T setMaxSpoutPending(Number val); 28 | T setNumTasks(Number val); 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/FailedException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | public class FailedException extends RuntimeException { 21 | public FailedException() { 22 | super(); 23 | } 24 | 25 | public FailedException(String msg) { 26 | super(msg); 27 | } 28 | 29 | public FailedException(String msg, Throwable cause) { 30 | super(msg, cause); 31 | } 32 | 33 | public FailedException(Throwable cause) { 34 | super(cause); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/IBasicOutputCollector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | import java.util.List; 21 | 22 | public interface IBasicOutputCollector { 23 | List emit(String streamId, List tuple); 24 | void emitDirect(int taskId, String streamId, List tuple); 25 | void reportError(Throwable t); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/IRichBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | import backtype.storm.task.IBolt; 21 | 22 | /** 23 | * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces 24 | * to use to implement components of the topology. 25 | * 26 | */ 27 | public interface IRichBolt extends IBolt, IComponent { 28 | 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/IRichSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | import backtype.storm.spout.ISpout; 21 | 22 | /** 23 | * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces 24 | * to use to implement components of the topology. 25 | * 26 | */ 27 | public interface IRichSpout extends ISpout, IComponent { 28 | 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/IRichStateSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | import backtype.storm.state.IStateSpout; 21 | 22 | 23 | public interface IRichStateSpout extends IStateSpout, IComponent { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/OutputFieldsDeclarer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | import backtype.storm.tuple.Fields; 21 | 22 | 23 | public interface OutputFieldsDeclarer { 24 | /** 25 | * Uses default stream id. 26 | */ 27 | public void declare(Fields fields); 28 | public void declare(boolean direct, Fields fields); 29 | 30 | public void declareStream(String streamId, Fields fields); 31 | public void declareStream(String streamId, boolean direct, Fields fields); 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/SpoutDeclarer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology; 19 | 20 | public interface SpoutDeclarer extends ComponentConfigurationDeclarer { 21 | 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BaseBasicBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.task.TopologyContext; 21 | import backtype.storm.topology.IBasicBolt; 22 | import java.util.Map; 23 | 24 | public abstract class BaseBasicBolt extends BaseComponent implements IBasicBolt { 25 | 26 | @Override 27 | public void prepare(Map stormConf, TopologyContext context) { 28 | } 29 | 30 | @Override 31 | public void cleanup() { 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BaseBatchBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.coordination.IBatchBolt; 21 | import java.util.Map; 22 | 23 | public abstract class BaseBatchBolt extends BaseComponent implements IBatchBolt { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BaseComponent.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.topology.IComponent; 21 | import java.util.Map; 22 | 23 | public abstract class BaseComponent implements IComponent { 24 | @Override 25 | public Map getComponentConfiguration() { 26 | return null; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; 21 | 22 | 23 | public abstract class BaseOpaquePartitionedTransactionalSpout extends BaseComponent implements IOpaquePartitionedTransactionalSpout { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; 21 | import java.util.Map; 22 | 23 | public abstract class BasePartitionedTransactionalSpout extends BaseComponent implements IPartitionedTransactionalSpout { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BaseRichBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.topology.IRichBolt; 21 | 22 | public abstract class BaseRichBolt extends BaseComponent implements IRichBolt { 23 | @Override 24 | public void cleanup() { 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.transactional.TransactionAttempt; 21 | 22 | public abstract class BaseTransactionalBolt extends BaseBatchBolt { 23 | 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.topology.base; 19 | 20 | import backtype.storm.transactional.ITransactionalSpout; 21 | import java.util.Map; 22 | 23 | public abstract class BaseTransactionalSpout extends BaseComponent implements ITransactionalSpout { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/transactional/ICommitter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.transactional; 19 | 20 | /** 21 | * This marks an IBatchBolt within a transactional topology as a committer. This causes the 22 | * finishBatch method to be called in order of the transactions. 23 | */ 24 | public interface ICommitter { 25 | 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/transactional/ICommitterTransactionalSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.transactional; 19 | 20 | import backtype.storm.task.TopologyContext; 21 | import java.util.Map; 22 | 23 | 24 | public interface ICommitterTransactionalSpout extends ITransactionalSpout { 25 | public interface Emitter extends ITransactionalSpout.Emitter { 26 | void commit(TransactionAttempt attempt); 27 | } 28 | 29 | @Override 30 | public Emitter getEmitter(Map conf, TopologyContext context); 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/tuple/Values.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.tuple; 19 | 20 | import java.util.ArrayList; 21 | 22 | /** 23 | * A convenience class for making tuple values using new Values("field1", 2, 3) 24 | * syntax. 25 | */ 26 | public class Values extends ArrayList{ 27 | public Values() { 28 | 29 | } 30 | 31 | public Values(Object... vals) { 32 | super(vals.length); 33 | for(Object o: vals) { 34 | add(o); 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.ui; 19 | 20 | public class InvalidRequestException extends Exception { 21 | 22 | public InvalidRequestException() { 23 | super(); 24 | } 25 | 26 | public InvalidRequestException(String msg) { 27 | super(msg); 28 | } 29 | 30 | public InvalidRequestException(String msg, Throwable cause) { 31 | super(msg, cause); 32 | } 33 | 34 | public InvalidRequestException(Throwable cause) { 35 | super(cause); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/utils/ClojureTimerTask.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.utils; 19 | 20 | import clojure.lang.IFn; 21 | import java.util.TimerTask; 22 | 23 | public class ClojureTimerTask extends TimerTask { 24 | IFn _afn; 25 | 26 | public ClojureTimerTask(IFn afn) { 27 | super(); 28 | _afn = afn; 29 | } 30 | 31 | @Override 32 | public void run() { 33 | _afn.run(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/utils/Container.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.utils; 19 | 20 | import java.io.Serializable; 21 | 22 | public class Container implements Serializable { 23 | public Object object; 24 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/backtype/storm/utils/MutableObject.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package backtype.storm.utils; 19 | 20 | public class MutableObject { 21 | Object o = null; 22 | 23 | public MutableObject() { 24 | 25 | } 26 | 27 | public MutableObject(Object o) { 28 | this.o = o; 29 | } 30 | 31 | public void setObject(Object o) { 32 | this.o = o; 33 | } 34 | 35 | public Object getObject() { 36 | return o; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/JoinType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident; 19 | 20 | import java.util.Arrays; 21 | import java.util.List; 22 | 23 | public enum JoinType { 24 | INNER, 25 | OUTER; 26 | 27 | public static List mixed(JoinType... types) { 28 | return Arrays.asList(types); 29 | } 30 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/fluent/GlobalAggregationScheme.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.fluent; 19 | 20 | import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; 21 | 22 | 23 | public interface GlobalAggregationScheme { 24 | IAggregatableStream aggPartition(S stream); // how to partition for second stage of aggregation 25 | BatchToPartition singleEmitPartitioner(); // return null if it's not single emit 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/fluent/IChainedAggregatorDeclarer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.fluent; 19 | 20 | import storm.trident.Stream; 21 | 22 | public interface IChainedAggregatorDeclarer { 23 | Stream chainEnd(); 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/fluent/UniqueIdGen.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.fluent; 19 | 20 | public class UniqueIdGen { 21 | int _streamCounter = 0; 22 | 23 | public String getUniqueStreamId() { 24 | _streamCounter++; 25 | return "s" + _streamCounter; 26 | } 27 | 28 | int _stateCounter = 0; 29 | 30 | public String getUniqueStateId() { 31 | _stateCounter++; 32 | return "state" + _stateCounter; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/Aggregator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import storm.trident.tuple.TridentTuple; 21 | 22 | public interface Aggregator extends Operation { 23 | T init(Object batchId, TridentCollector collector); 24 | void aggregate(T val, TridentTuple tuple, TridentCollector collector); 25 | void complete(T val, TridentCollector collector); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/Assembly.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import storm.trident.Stream; 21 | 22 | 23 | public interface Assembly { 24 | Stream apply(Stream input); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/BaseAggregator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | 21 | public abstract class BaseAggregator extends BaseOperation implements Aggregator { 22 | 23 | } 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/BaseFilter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | 21 | public abstract class BaseFilter extends BaseOperation implements Filter { 22 | 23 | } 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/BaseFunction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | 21 | public abstract class BaseFunction extends BaseOperation implements Function { 22 | 23 | } 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/BaseMultiReducer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import java.util.Map; 21 | 22 | public abstract class BaseMultiReducer implements MultiReducer { 23 | 24 | @Override 25 | public void prepare(Map conf, TridentMultiReducerContext context) { 26 | } 27 | 28 | 29 | @Override 30 | public void cleanup() { 31 | } 32 | 33 | } 34 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/BaseOperation.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import java.util.Map; 21 | 22 | public class BaseOperation implements Operation { 23 | 24 | @Override 25 | public void prepare(Map conf, TridentOperationContext context) { 26 | } 27 | 28 | @Override 29 | public void cleanup() { 30 | } 31 | 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/CombinerAggregator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import java.io.Serializable; 21 | import storm.trident.tuple.TridentTuple; 22 | 23 | // doesn't manipulate tuples (lists of stuff) so that things like aggregating into 24 | // cassandra is cleaner (don't need lists everywhere, just store the single value there) 25 | public interface CombinerAggregator extends Serializable { 26 | T init(TridentTuple tuple); 27 | T combine(T val1, T val2); 28 | T zero(); 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/EachOperation.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | public interface EachOperation extends Operation { 21 | 22 | } 23 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/Filter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import storm.trident.tuple.TridentTuple; 21 | 22 | 23 | public interface Filter extends EachOperation { 24 | boolean isKeep(TridentTuple tuple); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/Function.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import storm.trident.tuple.TridentTuple; 21 | 22 | public interface Function extends EachOperation { 23 | void execute(TridentTuple tuple, TridentCollector collector); 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/MultiReducer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import java.io.Serializable; 21 | import java.util.Map; 22 | import storm.trident.tuple.TridentTuple; 23 | 24 | 25 | public interface MultiReducer extends Serializable { 26 | void prepare(Map conf, TridentMultiReducerContext context); 27 | T init(TridentCollector collector); 28 | void execute(T state, int streamIndex, TridentTuple input, TridentCollector collector); 29 | void complete(T state, TridentCollector collector); 30 | void cleanup(); 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/Operation.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import java.io.Serializable; 21 | import java.util.Map; 22 | 23 | public interface Operation extends Serializable { 24 | void prepare(Map conf, TridentOperationContext context); 25 | void cleanup(); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/ReducerAggregator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import java.io.Serializable; 21 | import storm.trident.tuple.TridentTuple; 22 | 23 | public interface ReducerAggregator extends Serializable { 24 | T init(); 25 | T reduce(T curr, TridentTuple tuple); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/TridentCollector.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation; 19 | 20 | import java.util.List; 21 | 22 | 23 | public interface TridentCollector { 24 | void emit(List values); 25 | void reportError(Throwable t); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/builtin/FilterNull.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation.builtin; 19 | 20 | import storm.trident.operation.BaseFilter; 21 | import storm.trident.tuple.TridentTuple; 22 | 23 | public class FilterNull extends BaseFilter { 24 | @Override 25 | public boolean isKeep(TridentTuple tuple) { 26 | for(Object o: tuple) { 27 | if(o==null) return false; 28 | } 29 | return true; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/impl/GlobalBatchToPartition.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation.impl; 19 | 20 | 21 | public class GlobalBatchToPartition implements SingleEmitAggregator.BatchToPartition { 22 | 23 | @Override 24 | public int partitionIndex(Object batchId, int numPartitions) { 25 | // TODO: take away knowledge of storm's internals here 26 | return 0; 27 | } 28 | 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/impl/IndexHashBatchToPartition.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation.impl; 19 | 20 | import storm.trident.partition.IndexHashGrouping; 21 | 22 | public class IndexHashBatchToPartition implements SingleEmitAggregator.BatchToPartition { 23 | 24 | @Override 25 | public int partitionIndex(Object batchId, int numPartitions) { 26 | return IndexHashGrouping.objectToIndex(batchId, numPartitions); 27 | } 28 | 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/operation/impl/Result.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.operation.impl; 19 | 20 | public class Result { 21 | public Object obj; 22 | 23 | @Override 24 | public String toString() { 25 | return "" + obj; 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/planner/NodeStateInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.planner; 19 | 20 | import java.io.Serializable; 21 | import storm.trident.state.StateSpec; 22 | 23 | public class NodeStateInfo implements Serializable { 24 | public String id; 25 | public StateSpec spec; 26 | 27 | public NodeStateInfo(String id, StateSpec spec) { 28 | this.id = id; 29 | this.spec = spec; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/planner/ProcessorContext.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.planner; 19 | 20 | 21 | public class ProcessorContext { 22 | public Object batchId; 23 | public Object[] state; 24 | 25 | public ProcessorContext(Object batchId, Object[] state) { 26 | this.batchId = batchId; 27 | this.state = state; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/planner/TupleReceiver.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.planner; 19 | 20 | import storm.trident.tuple.TridentTuple; 21 | 22 | 23 | public interface TupleReceiver { 24 | //streaId indicates where tuple came from 25 | void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple); 26 | 27 | } 28 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/spout/IBatchID.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.spout; 19 | 20 | 21 | public interface IBatchID { 22 | Object getId(); 23 | int getAttemptId(); 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/spout/ICommitterTridentSpout.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.spout; 19 | 20 | import backtype.storm.task.TopologyContext; 21 | import storm.trident.topology.TransactionAttempt; 22 | import java.util.Map; 23 | 24 | public interface ICommitterTridentSpout extends ITridentSpout { 25 | public interface Emitter extends ITridentSpout.Emitter { 26 | void commit(TransactionAttempt attempt); 27 | } 28 | 29 | @Override 30 | public Emitter getEmitter(String txStateId, Map conf, TopologyContext context); 31 | } -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/spout/ISpoutPartition.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.spout; 19 | 20 | public interface ISpoutPartition { 21 | /** 22 | * This is used as a Zookeeper node path for storing metadata. 23 | */ 24 | String getId(); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/BaseQueryFunction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | import storm.trident.operation.BaseOperation; 21 | 22 | 23 | public abstract class BaseQueryFunction extends BaseOperation implements QueryFunction { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/BaseStateUpdater.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | import storm.trident.operation.BaseOperation; 21 | 22 | 23 | public abstract class BaseStateUpdater extends BaseOperation implements StateUpdater { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/ITupleCollection.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | import java.util.Iterator; 21 | import java.util.List; 22 | 23 | /* Container of a collection of tuples */ 24 | public interface ITupleCollection { 25 | public Iterator> getTuples(); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/QueryFunction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | import java.util.List; 21 | import storm.trident.operation.EachOperation; 22 | import storm.trident.operation.TridentCollector; 23 | import storm.trident.tuple.TridentTuple; 24 | 25 | public interface QueryFunction extends EachOperation { 26 | List batchRetrieve(S state, List args); 27 | void execute(TridentTuple tuple, T result, TridentCollector collector); 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/ReadOnlyState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | public class ReadOnlyState implements State { 21 | 22 | @Override 23 | public void beginCommit(Long txid) { 24 | throw new UnsupportedOperationException("This state is read-only and does not support updates"); 25 | } 26 | 27 | @Override 28 | public void commit(Long txid) { 29 | throw new UnsupportedOperationException("This state is read-only and does not support updates"); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/Serializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | import java.io.Serializable; 21 | 22 | 23 | public interface Serializer extends Serializable { 24 | byte[] serialize(T obj); 25 | T deserialize(byte[] b); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/StateFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | import backtype.storm.task.IMetricsContext; 21 | import java.io.Serializable; 22 | import java.util.Map; 23 | 24 | public interface StateFactory extends Serializable { 25 | State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/StateSpec.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | import java.io.Serializable; 21 | 22 | 23 | public class StateSpec implements Serializable { 24 | public StateFactory stateFactory; 25 | public Integer requiredNumPartitions = null; 26 | 27 | public StateSpec(StateFactory stateFactory) { 28 | this.stateFactory = stateFactory; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/StateType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | 21 | public enum StateType { 22 | NON_TRANSACTIONAL, 23 | TRANSACTIONAL, 24 | OPAQUE 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/ValueUpdater.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state; 19 | 20 | 21 | public interface ValueUpdater { 22 | T update(T stored); 23 | } 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/map/IBackingMap.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state.map; 19 | 20 | import java.util.List; 21 | 22 | 23 | public interface IBackingMap { 24 | List multiGet(List> keys); 25 | void multiPut(List> keys, List vals); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/map/MapState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state.map; 19 | 20 | import java.util.List; 21 | import storm.trident.state.ValueUpdater; 22 | 23 | public interface MapState extends ReadOnlyMapState { 24 | List multiUpdate(List> keys, List updaters); 25 | void multiPut(List> keys, List vals); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/map/ReadOnlyMapState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state.map; 19 | 20 | import java.util.List; 21 | import storm.trident.state.State; 22 | 23 | public interface ReadOnlyMapState extends State { 24 | // certain states might only accept one-tuple keys - those should just throw an error 25 | List multiGet(List> keys); 26 | } 27 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/map/RemovableMapState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state.map; 19 | 20 | import java.util.List; 21 | import storm.trident.state.State; 22 | 23 | public interface RemovableMapState extends State { 24 | void multiRemove(List> keys); 25 | } 26 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/snapshot/ReadOnlySnapshottable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state.snapshot; 19 | 20 | import storm.trident.state.State; 21 | 22 | public interface ReadOnlySnapshottable extends State { 23 | T get(); 24 | } 25 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/state/snapshot/Snapshottable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.state.snapshot; 19 | 20 | import storm.trident.state.ValueUpdater; 21 | 22 | 23 | // used by Stream#persistentAggregate 24 | public interface Snapshottable extends ReadOnlySnapshottable { 25 | T update(ValueUpdater updater); 26 | void set(T o); 27 | } 28 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/testing/IFeeder.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.testing; 19 | 20 | 21 | public interface IFeeder { 22 | void feed(Object tuples); 23 | } 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/testing/StringLength.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.testing; 19 | 20 | import backtype.storm.tuple.Values; 21 | import storm.trident.operation.BaseFunction; 22 | import storm.trident.operation.TridentCollector; 23 | import storm.trident.tuple.TridentTuple; 24 | 25 | public class StringLength extends BaseFunction { 26 | 27 | @Override 28 | public void execute(TridentTuple tuple, TridentCollector collector) { 29 | collector.emit(new Values(tuple.getString(0).length())); 30 | } 31 | 32 | } 33 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/testing/TrueFilter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.testing; 19 | 20 | import storm.trident.operation.BaseFilter; 21 | import storm.trident.tuple.TridentTuple; 22 | 23 | public class TrueFilter extends BaseFilter { 24 | 25 | @Override 26 | public boolean isKeep(TridentTuple tuple) { 27 | return true; 28 | } 29 | 30 | } 31 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/topology/BatchInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.topology; 19 | 20 | import storm.trident.spout.IBatchID; 21 | 22 | 23 | public class BatchInfo { 24 | public IBatchID batchId; 25 | public Object state; 26 | public String batchGroup; 27 | 28 | public BatchInfo(String batchGroup, IBatchID batchId, Object state) { 29 | this.batchGroup = batchGroup; 30 | this.batchId = batchId; 31 | this.state = state; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/tuple/TridentTuple.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.tuple; 19 | 20 | import backtype.storm.tuple.ITuple; 21 | 22 | import java.io.Serializable; 23 | import java.util.List; 24 | import java.util.Map; 25 | 26 | public interface TridentTuple extends ITuple, List { 27 | 28 | public static interface Factory extends Serializable { 29 | Map getFieldIndex(); 30 | List getOutputFields(); 31 | int numDelegates(); 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/util/ErrorEdgeFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.util; 19 | 20 | import java.io.Serializable; 21 | import org.jgrapht.EdgeFactory; 22 | 23 | public class ErrorEdgeFactory implements EdgeFactory, Serializable { 24 | @Override 25 | public Object createEdge(Object v, Object v1) { 26 | throw new RuntimeException("Edges should be made explicitly"); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/jvm/storm/trident/util/LRUMap.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package storm.trident.util; 19 | 20 | import java.util.LinkedHashMap; 21 | import java.util.Map; 22 | 23 | public class LRUMap extends LinkedHashMap { 24 | private int _maxSize; 25 | 26 | public LRUMap(int maxSize) { 27 | super(maxSize + 1, 1.0f, true); 28 | _maxSize = maxSize; 29 | } 30 | 31 | @Override 32 | protected boolean removeEldestEntry(final Map.Entry eldest) { 33 | return size() > _maxSize; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/py/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bdatadev/storm-src/906eb6a20af925458b434fa901c141f6407d5dce/apache-storm-0.9.4/storm-core/src/py/__init__.py -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/py/storm/__init__.py: -------------------------------------------------------------------------------- 1 | __all__ = ['ttypes', 'constants', 'Nimbus', 'DistributedRPC', 'DistributedRPCInvocations'] 2 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/py/storm/constants.py: -------------------------------------------------------------------------------- 1 | # 2 | # Autogenerated by Thrift Compiler (0.7.0) 3 | # 4 | # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING 5 | # 6 | 7 | from thrift.Thrift import * 8 | from ttypes import * 9 | 10 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/ui/public/images/spinner.gif: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bdatadev/storm-src/906eb6a20af925458b434fa901c141f6407d5dce/apache-storm-0.9.4/storm-core/src/ui/public/images/spinner.gif -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/src/ui/public/templates/json-error-template.html: -------------------------------------------------------------------------------- 1 | 17 | 21 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/test/clj/backtype/storm/security/auth/jaas_digest.conf: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /* This sample file illustrates how Digest authentication should be configured 20 | */ 21 | StormServer { 22 | org.apache.zookeeper.server.auth.DigestLoginModule required 23 | user_super="adminsecret" 24 | user_bob="bobsecret"; 25 | }; 26 | StormClient { 27 | org.apache.zookeeper.server.auth.DigestLoginModule required 28 | username="bob" 29 | password="bobsecret"; 30 | }; -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/test/clj/backtype/storm/security/auth/jaas_digest_bad_password.conf: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /* This sample file containes incorrect password of a user. 20 | We use this file for negative test. 21 | */ 22 | StormServer { 23 | org.apache.zookeeper.server.auth.DigestLoginModule required 24 | user_super="adminsecret" 25 | user_bob="bobsecret"; 26 | }; 27 | StormClient { 28 | org.apache.zookeeper.server.auth.DigestLoginModule required 29 | username="bob" 30 | password="bad_password"; 31 | }; -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/test/clj/backtype/storm/security/auth/jaas_digest_missing_client.conf: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | StormServer { 20 | org.apache.zookeeper.server.auth.DigestLoginModule required 21 | user_super="adminsecret" 22 | user_bob="bobsecret"; 23 | }; 24 | -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-core/test/clj/backtype/storm/security/auth/jaas_digest_unknown_user.conf: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /* This sample file containes an unauthorized user. 20 | We use this file for negative test. 21 | */ 22 | StormServer { 23 | org.apache.zookeeper.server.auth.DigestLoginModule required 24 | user_super="adminsecret" 25 | user_bob="bobsecret"; 26 | }; 27 | StormClient { 28 | org.apache.zookeeper.server.auth.DigestLoginModule required 29 | username="unknown_user" 30 | password="some_password"; 31 | }; -------------------------------------------------------------------------------- /apache-storm-0.9.4/storm-dist/binary/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Storm 2 | Copyright 2014 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | 7 | This product includes software developed by Yahoo! Inc. (www.yahoo.com) 8 | Copyright (c) 2012-2014 Yahoo! Inc. 9 | 10 | YAML support provided by snakeyaml (http://code.google.com/p/snakeyaml/). 11 | Copyright (c) 2008-2010 Andrey Somov 12 | 13 | The Netty transport uses Netty 14 | (https://netty.io/) 15 | Copyright (C) 2011 The Netty Project 16 | 17 | This product uses LMAX Disruptor 18 | (http://lmax-exchange.github.io/disruptor/) 19 | Copyright 2011 LMAX Ltd. 20 | 21 | This product includes the Jetty HTTP server 22 | (http://jetty.codehaus.org/jetty/). 23 | Copyright 1995-2006 Mort Bay Consulting Pty Ltd 24 | 25 | JSON (de)serialization by json-simple from 26 | (http://code.google.com/p/json-simple). 27 | Copyright (C) 2009 Fang Yidong and Chris Nokleberg 28 | 29 | Alternative collection types provided by google-collections from 30 | http://code.google.com/p/google-collections/. 31 | Copyright (C) 2007 Google Inc. 32 | --------------------------------------------------------------------------------