diff --git a/CHANGELOG.md b/CHANGELOG.md index c6387df9f..79b08eb0c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,23 @@ +## 0.9.2-incubating (unreleased) + * STORM-173: Treat command line "-c" option number config values as such + * STORM-194: Support list of strings in *.worker.childopts, handle spaces + * STORM-288: Fixes version spelling in pom.xml + * STORM-208: Add storm-kafka as an external module + * STORM-285: Fix storm-core shade plugin config + * STORM-12: reduce thread usage of netty transport + * STORM-281: fix and issue with config parsing that could lead to leaking file descriptors + * STORM-196: When JVM_OPTS are set, storm jar fails to detect storm.jar from environment + * STORM-260: Fix a potential race condition with simulated time in Storm's unit tests + * STORM-258: Update commons-io version to 2.4 + * STORM-270: don't package .clj files in release jars. + * STORM-273: Error while running storm topologies on Windows using "storm jar" + * STROM-247: Replace links to github resources in storm script + * STORM-263: Update Kryo version to 2.21+ + * STORM-187: Fix Netty error "java.lang.IllegalArgumentException: timeout value is negative" + * STORM-186: fix float secs to millis long convertion + * STORM-70: Upgrade to ZK-3.4.5 and curator-1.3.3 + * STORM-146: Unit test regression when storm is compiled with 3.4.5 zookeeper + ## 0.9.1-incubating * Fix to prevent Nimbus from hanging if random data is sent to nimbus thrift port * Improved support for running on Windows platforms diff --git a/NOTICE b/NOTICE index 4f3e648d8..637e578c5 100644 --- a/NOTICE +++ b/NOTICE @@ -1,13 +1,8 @@ Apache Storm -Copyright 2013 The Apache Software Foundation +Copyright 2014 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by Nathan Marz -Copyright 2011-2013 Nathan Marz - - This product includes software developed by Yahoo! Inc. (www.yahoo.com) -Copyright © 2012-2013 Yahoo! Inc. All rights reserved. +Copyright (c) 2012-2014 Yahoo! Inc. \ No newline at end of file diff --git a/README.markdown b/README.markdown index 525463922..46ff724f7 100644 --- a/README.markdown +++ b/README.markdown @@ -71,6 +71,9 @@ under the License. * Flip Kromer ([@mrflip](https://github.com/mrflip)) * David Lao ([@davidlao2k](https://github.com/davidlao2k)) * P. Taylor Goetz ([@ptgoetz](https://github.com/ptgoetz)) +* Derek Dagit ([@d2r](https://github.com/d2r)) +* Robert Evans ([@revans2](https://github.com/revans2)) + ## Contributors @@ -105,9 +108,7 @@ under the License. * Bryan Peterson ([@Lazyshot](https://github.com/Lazyshot)) * Sam Ritchie ([@sritchie](https://github.com/sritchie)) * Stuart Anderson ([@emblem](https://github.com/emblem)) -* Robert Evans ([@revans2](https://github.com/revans2)) * Lorcan Coyle ([@lorcan](https://github.com/lorcan)) -* Derek Dagit ([@d2r](https://github.com/d2r)) * Andrew Olson ([@noslowerdna](https://github.com/noslowerdna)) * Gavin Li ([@lyogavin](https://github.com/lyogavin)) * Tudor Scurtu ([@tscurtu](https://github.com/tscurtu)) diff --git a/bin/storm b/bin/storm index aee46cbfa..bb92f1cf0 100755 --- a/bin/storm +++ b/bin/storm @@ -22,6 +22,7 @@ import random import subprocess as sub import getopt import re +import shlex def identity(x): return x @@ -45,7 +46,7 @@ if (not os.path.isfile(USER_CONF_DIR + "/storm.yaml")): USER_CONF_DIR = CLUSTER_CONF_DIR CONFIG_OPTS = [] CONFFILE = "" -JAR_JVM_OPTS = os.getenv('STORM_JAR_JVM_OPTS', '') +JAR_JVM_OPTS = shlex.split(os.getenv('STORM_JAR_JVM_OPTS', '')) def get_config_opts(): @@ -147,7 +148,7 @@ def jar(jarfile, klass, *args): Runs the main method of class with the specified arguments. The storm jars and configs in ~/.storm are put on the classpath. The process is configured so that StormSubmitter - (http://nathanmarz.github.com/storm/doc/backtype/storm/StormSubmitter.html) + (http://storm.incubator.apache.org/apidocs/backtype/storm/StormSubmitter.html) will upload the jar at topology-jar-path when the topology is submitted. """ exec_storm_class( @@ -155,7 +156,7 @@ def jar(jarfile, klass, *args): jvmtype="-client", extrajars=[jarfile, USER_CONF_DIR, STORM_DIR + "/bin"], args=args, - jvmopts=[' '.join(filter(None, [JAR_JVM_OPTS, "-Dstorm.jar=" + jarfile]))]) + jvmopts=JAR_JVM_OPTS + ["-Dstorm.jar=" + jarfile]) def kill(*args): """Syntax: [storm kill topology-name [-w wait-time-secs]] @@ -262,7 +263,7 @@ def nimbus(klass="backtype.storm.daemon.nimbus"): supervision with a tool like daemontools or monit. See Setting up a Storm cluster for more information. - (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster) + (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster) """ cppaths = [CLUSTER_CONF_DIR] jvmopts = parse_args(confvalue("nimbus.childopts", cppaths)) + [ @@ -282,7 +283,7 @@ def supervisor(klass="backtype.storm.daemon.supervisor"): under supervision with a tool like daemontools or monit. See Setting up a Storm cluster for more information. - (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster) + (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster) """ cppaths = [CLUSTER_CONF_DIR] jvmopts = parse_args(confvalue("supervisor.childopts", cppaths)) + [ @@ -303,7 +304,7 @@ def ui(): should be run under supervision with a tool like daemontools or monit. See Setting up a Storm cluster for more information. - (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster) + (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster) """ cppaths = [CLUSTER_CONF_DIR] jvmopts = parse_args(confvalue("ui.childopts", cppaths)) + [ @@ -324,7 +325,7 @@ def logviewer(): tool like daemontools or monit. See Setting up a Storm cluster for more information. - (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster) + (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster) """ cppaths = [CLUSTER_CONF_DIR] jvmopts = parse_args(confvalue("logviewer.childopts", cppaths)) + [ @@ -344,7 +345,7 @@ def drpc(): with a tool like daemontools or monit. See Distributed RPC for more information. - (https://github.com/nathanmarz/storm/wiki/Distributed-RPC) + (http://storm.incubator.apache.org/documentation/Distributed-RPC) """ cppaths = [CLUSTER_CONF_DIR] jvmopts = parse_args(confvalue("drpc.childopts", cppaths)) + [ @@ -392,7 +393,7 @@ def print_commands(): """Print all client commands and link to documentation""" print "Commands:\n\t", "\n\t".join(sorted(COMMANDS.keys())) print "\nHelp:", "\n\thelp", "\n\thelp " - print "\nDocumentation for the storm client can be found at https://github.com/nathanmarz/storm/wiki/Command-line-client\n" + print "\nDocumentation for the storm client can be found at http://storm.incubator.apache.org/documentation/Command-line-client.html\n" print "Configs can be overridden using one or more -c flags, e.g. \"storm list -c nimbus.host=nimbus.mycompany.com\"\n" def print_usage(command=None): diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 2dbba24c9..66b2d413e 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -35,6 +35,7 @@ storm.zookeeper.retry.intervalceiling.millis: 30000 storm.cluster.mode: "distributed" # can be distributed or local storm.local.mode.zmq: false storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin" +storm.multilang.serializer: "backtype.storm.multilang.JsonSerializer" storm.messaging.transport: "backtype.storm.messaging.netty.Context" ### nimbus.* configs are for the master diff --git a/examples/storm-starter/README.markdown b/examples/storm-starter/README.markdown new file mode 100644 index 000000000..a85f5991c --- /dev/null +++ b/examples/storm-starter/README.markdown @@ -0,0 +1,108 @@ +# Example Storm Topologies + +Learn to use Storm! + +--- + +Table of Contents + +* Getting started +* Using storm-starter with Maven +* Using storm-starter with IntelliJ IDEA + +--- + + + + +# Getting started + +## Prerequisites + +First, you need `java` and `git` installed and in your user's `PATH`. Also, two of the examples in storm-starter +require Python and Ruby. + +Next, make sure you have the storm-starter code available on your machine. Git/GitHub beginners may want to use the +following command to download the latest storm-starter code and change to the new directory that contains the downloaded +code. + + $ git clone git://github.com/apache/incubator-storm.git && cd incubator-storm/examples/storm-starter + + +## storm-starter overview + +storm-starter contains a variety of examples of using Storm. If this is your first time working with Storm, check out +these topologies first: + +1. [ExclamationTopology](src/jvm/storm/starter/ExclamationTopology.java): Basic topology written in all Java +2. [WordCountTopology](src/jvm/storm/starter/WordCountTopology.java): Basic topology that makes use of multilang by + implementing one bolt in Python +3. [ReachTopology](src/jvm/storm/starter/ReachTopology.java): Example of complex DRPC on top of Storm + +After you have familiarized yourself with these topologies, take a look at the other topopologies in +[src/jvm/storm/starter/](src/jvm/storm/starter/) such as [RollingTopWords](src/jvm/storm/starter/RollingTopWords.java) +for more advanced implementations. + +If you want to learn more about how Storm works, please head over to the +[Storm project page](http://storm.incubator.apache.org). + + + + +# Using storm-starter with Maven + +## Install Maven + +Install [Maven](http://maven.apache.org/) (preferably version 3.x) by following +the [Maven installation instructions](http://maven.apache.org/download.cgi). + + +## Running topologies with Maven + +storm-starter topologies can be run with the maven-exec-plugin. For example, to +compile and run `WordCountTopology` in local mode, use the command: + + $ mvn compile exec:java -Dstorm.topology=storm.starter.WordCountTopology + +You can also run clojure topologies with Maven: + + $ mvn compile exec:java -Dstorm.topology=storm.starter.clj.word_count + +## Packaging storm-starter for use on a Storm cluster + +You can package a jar suitable for submitting to a Storm cluster with the command: + + $ mvn package + +This will package your code and all the non-Storm dependencies into a single "uberjar" at the path +`target/storm-starter-{version}-jar-with-dependencies.jar`. + + +## Running unit tests + +Use the following Maven command to run the unit tests that ship with storm-starter. Unfortunately `lein test` does not +yet run the included unit tests. + + $ mvn test + + + + +# Using storm-starter with IntelliJ IDEA + +## Importing storm-starter as a project in IDEA + +The following instructions will import storm-starter as a new project in IntelliJ IDEA. + + +* Open _File > Import Project..._ and navigate to the storm-starter directory of your storm clone (e.g. + `~/git/incubator-storm/examples/storm-starter`). +* Select _Import project from external model_, select "Maven", and click _Next_. +* In the following screen, enable the checkbox _Import Maven projects automatically_. Leave all other values at their + defaults. Click _Next_. +* Click _Next_ on the following screen about selecting Maven projects to import. +* Select the JDK to be used by IDEA for storm-starter, then click _Next_. + * At the time of this writing you should use JDK 6. + * It is strongly recommended to use Sun/Oracle JDK 6 rather than OpenJDK 6. +* You may now optionally change the name of the project in IDEA. The default name suggested by IDEA is "storm-starter". + Click _Finish_ once you are done. diff --git a/examples/storm-starter/multilang/resources/splitsentence.py b/examples/storm-starter/multilang/resources/splitsentence.py new file mode 100644 index 000000000..300105f2c --- /dev/null +++ b/examples/storm-starter/multilang/resources/splitsentence.py @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import storm + +class SplitSentenceBolt(storm.BasicBolt): + def process(self, tup): + words = tup.values[0].split(" ") + for word in words: + storm.emit([word]) + +SplitSentenceBolt().run() \ No newline at end of file diff --git a/examples/storm-starter/multilang/resources/splitsentence.rb b/examples/storm-starter/multilang/resources/splitsentence.rb new file mode 100644 index 000000000..6d1d2463d --- /dev/null +++ b/examples/storm-starter/multilang/resources/splitsentence.rb @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +require "./storm" + +class SplitSentenceBolt < Storm::Bolt + def process(tup) + tup.values[0].split(" ").each do |word| + emit([word]) + end + end +end + +SplitSentenceBolt.new.run \ No newline at end of file diff --git a/examples/storm-starter/multilang/resources/storm.py b/examples/storm-starter/multilang/resources/storm.py new file mode 100644 index 000000000..4bc9e8248 --- /dev/null +++ b/examples/storm-starter/multilang/resources/storm.py @@ -0,0 +1,221 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import sys +import os +import traceback +from collections import deque + +try: + import simplejson as json +except ImportError: + import json + +json_encode = lambda x: json.dumps(x) +json_decode = lambda x: json.loads(x) + +#reads lines and reconstructs newlines appropriately +def readMsg(): + msg = "" + while True: + line = sys.stdin.readline()[0:-1] + if line == "end": + break + msg = msg + line + "\n" + return json_decode(msg[0:-1]) + +MODE = None +ANCHOR_TUPLE = None + +#queue up commands we read while trying to read taskids +pending_commands = deque() + +def readTaskIds(): + if pending_taskids: + return pending_taskids.popleft() + else: + msg = readMsg() + while type(msg) is not list: + pending_commands.append(msg) + msg = readMsg() + return msg + +#queue up taskids we read while trying to read commands/tuples +pending_taskids = deque() + +def readCommand(): + if pending_commands: + return pending_commands.popleft() + else: + msg = readMsg() + while type(msg) is list: + pending_taskids.append(msg) + msg = readMsg() + return msg + +def readTuple(): + cmd = readCommand() + return Tuple(cmd["id"], cmd["comp"], cmd["stream"], cmd["task"], cmd["tuple"]) + +def sendMsgToParent(msg): + print json_encode(msg) + print "end" + sys.stdout.flush() + +def sync(): + sendMsgToParent({'command':'sync'}) + +def sendpid(heartbeatdir): + pid = os.getpid() + sendMsgToParent({'pid':pid}) + open(heartbeatdir + "/" + str(pid), "w").close() + +def emit(*args, **kwargs): + __emit(*args, **kwargs) + return readTaskIds() + +def emitDirect(task, *args, **kwargs): + kwargs[directTask] = task + __emit(*args, **kwargs) + +def __emit(*args, **kwargs): + global MODE + if MODE == Bolt: + emitBolt(*args, **kwargs) + elif MODE == Spout: + emitSpout(*args, **kwargs) + +def emitBolt(tup, stream=None, anchors = [], directTask=None): + global ANCHOR_TUPLE + if ANCHOR_TUPLE is not None: + anchors = [ANCHOR_TUPLE] + m = {"command": "emit"} + if stream is not None: + m["stream"] = stream + m["anchors"] = map(lambda a: a.id, anchors) + if directTask is not None: + m["task"] = directTask + m["tuple"] = tup + sendMsgToParent(m) + +def emitSpout(tup, stream=None, id=None, directTask=None): + m = {"command": "emit"} + if id is not None: + m["id"] = id + if stream is not None: + m["stream"] = stream + if directTask is not None: + m["task"] = directTask + m["tuple"] = tup + sendMsgToParent(m) + +def ack(tup): + sendMsgToParent({"command": "ack", "id": tup.id}) + +def fail(tup): + sendMsgToParent({"command": "fail", "id": tup.id}) + +def log(msg): + sendMsgToParent({"command": "log", "msg": msg}) + +def initComponent(): + setupInfo = readMsg() + sendpid(setupInfo['pidDir']) + return [setupInfo['conf'], setupInfo['context']] + +class Tuple: + def __init__(self, id, component, stream, task, values): + self.id = id + self.component = component + self.stream = stream + self.task = task + self.values = values + + def __repr__(self): + return '<%s%s>' % ( + self.__class__.__name__, + ''.join(' %s=%r' % (k, self.__dict__[k]) for k in sorted(self.__dict__.keys()))) + +class Bolt: + def initialize(self, stormconf, context): + pass + + def process(self, tuple): + pass + + def run(self): + global MODE + MODE = Bolt + conf, context = initComponent() + self.initialize(conf, context) + try: + while True: + tup = readTuple() + self.process(tup) + except Exception, e: + log(traceback.format_exc(e)) + +class BasicBolt: + def initialize(self, stormconf, context): + pass + + def process(self, tuple): + pass + + def run(self): + global MODE + MODE = Bolt + global ANCHOR_TUPLE + conf, context = initComponent() + self.initialize(conf, context) + try: + while True: + tup = readTuple() + ANCHOR_TUPLE = tup + self.process(tup) + ack(tup) + except Exception, e: + log(traceback.format_exc(e)) + +class Spout: + def initialize(self, conf, context): + pass + + def ack(self, id): + pass + + def fail(self, id): + pass + + def nextTuple(self): + pass + + def run(self): + global MODE + MODE = Spout + conf, context = initComponent() + self.initialize(conf, context) + try: + while True: + msg = readCommand() + if msg["command"] == "next": + self.nextTuple() + if msg["command"] == "ack": + self.ack(msg["id"]) + if msg["command"] == "fail": + self.fail(msg["id"]) + sync() + except Exception, e: + log(traceback.format_exc(e)) diff --git a/examples/storm-starter/multilang/resources/storm.rb b/examples/storm-starter/multilang/resources/storm.rb new file mode 100644 index 000000000..985b4123c --- /dev/null +++ b/examples/storm-starter/multilang/resources/storm.rb @@ -0,0 +1,200 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +require "rubygems" +require "json" + +module Storm + module Protocol + class << self + attr_accessor :mode, :pending_taskids, :pending_commands + end + + self.pending_taskids = [] + self.pending_commands = [] + + def read_message + msg = "" + loop do + line = STDIN.readline.chomp + break if line == "end" + msg << line + msg << "\n" + end + JSON.parse msg.chomp + end + + def read_task_ids + Storm::Protocol.pending_taskids.shift || + begin + msg = read_message + until msg.is_a? Array + Storm::Protocol.pending_commands.push(msg) + msg = read_message + end + msg + end + end + + def read_command + Storm::Protocol.pending_commands.shift || + begin + msg = read_message + while msg.is_a? Array + Storm::Protocol.pending_taskids.push(msg) + msg = read_message + end + msg + end + end + + def send_msg_to_parent(msg) + puts msg.to_json + puts "end" + STDOUT.flush + end + + def sync + send_msg_to_parent({'command' => 'sync'}) + end + + def send_pid(heartbeat_dir) + pid = Process.pid + send_msg_to_parent({'pid' => pid}) + File.open("#{heartbeat_dir}/#{pid}", "w").close + end + + def emit_bolt(tup, args = {}) + stream = args[:stream] + anchors = args[:anchors] || args[:anchor] || [] + anchors = [anchors] unless anchors.is_a? Enumerable + direct = args[:direct_task] + m = {:command => :emit, :anchors => anchors.map(&:id), :tuple => tup} + m[:stream] = stream if stream + m[:task] = direct if direct + send_msg_to_parent m + read_task_ids unless direct + end + + def emit_spout(tup, args = {}) + stream = args[:stream] + id = args[:id] + direct = args[:direct_task] + m = {:command => :emit, :tuple => tup} + m[:id] = id if id + m[:stream] = stream if stream + m[:task] = direct if direct + send_msg_to_parent m + read_task_ids unless direct + end + + def emit(*args) + case Storm::Protocol.mode + when 'spout' + emit_spout(*args) + when 'bolt' + emit_bolt(*args) + end + end + + def ack(tup) + send_msg_to_parent :command => :ack, :id => tup.id + end + + def fail(tup) + send_msg_to_parent :command => :fail, :id => tup.id + end + + def log(msg) + send_msg_to_parent :command => :log, :msg => msg.to_s + end + + def handshake + setup_info = read_message + send_pid setup_info['pidDir'] + [setup_info['conf'], setup_info['context']] + end + end + + class Tuple + attr_accessor :id, :component, :stream, :task, :values + + def initialize(id, component, stream, task, values) + @id = id + @component = component + @stream = stream + @task = task + @values = values + end + + def self.from_hash(hash) + Tuple.new(*hash.values_at("id", "comp", "stream", "task", "tuple")) + end + end + + class Bolt + include Storm::Protocol + + def prepare(conf, context); end + + def process(tuple); end + + def run + Storm::Protocol.mode = 'bolt' + prepare(*handshake) + begin + while true + process Tuple.from_hash(read_command) + end + rescue Exception => e + log 'Exception in bolt: ' + e.message + ' - ' + e.backtrace.join('\n') + end + end + end + + class Spout + include Storm::Protocol + + def open(conf, context); end + + def nextTuple; end + + def ack(id); end + + def fail(id); end + + def run + Storm::Protocol.mode = 'spout' + open(*handshake) + + begin + while true + msg = read_command + case msg['command'] + when 'next' + nextTuple + when 'ack' + ack(msg['id']) + when 'fail' + fail(msg['id']) + end + sync + end + rescue Exception => e + log 'Exception in spout: ' + e.message + ' - ' + e.backtrace.join('\n') + end + end + end +end diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml new file mode 100644 index 000000000..3030a8fd7 --- /dev/null +++ b/examples/storm-starter/pom.xml @@ -0,0 +1,161 @@ + + + + 4.0.0 + + storm + org.apache.storm + 0.9.2-incubating-SNAPSHOT + ../../pom.xml + + + org.apache.storm + storm-starter + jar + + storm-starter + + + + + junit + junit + test + + + org.testng + testng + 6.8.5 + test + + + org.mockito + mockito-all + test + + + org.easytesting + fest-assert-core + 2.0M8 + test + + + org.jmock + jmock + 2.6.0 + test + + + org.apache.storm + storm-core + ${project.version} + + provided + + + commons-collections + commons-collections + 3.2.1 + + + com.google.guava + guava + + + + + src/jvm + test/jvm + + + ${basedir}/multilang + + + + + + + maven-assembly-plugin + + + jar-with-dependencies + + + + + + + + + + make-assembly + package + + single + + + + + + + com.theoryinpractise + clojure-maven-plugin + true + + + src/clj + + + + + compile + compile + + compile + + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + + exec + + + + + java + true + false + compile + ${storm.topology} + + + + + diff --git a/examples/storm-starter/src/clj/storm/starter/clj/word_count.clj b/examples/storm-starter/src/clj/storm/starter/clj/word_count.clj new file mode 100644 index 000000000..3b54ac81e --- /dev/null +++ b/examples/storm-starter/src/clj/storm/starter/clj/word_count.clj @@ -0,0 +1,95 @@ +;; Licensed to the Apache Software Foundation (ASF) under one +;; or more contributor license agreements. See the NOTICE file +;; distributed with this work for additional information +;; regarding copyright ownership. The ASF licenses this file +;; to you under the Apache License, Version 2.0 (the +;; "License"); you may not use this file except in compliance +;; with the License. You may obtain a copy of the License at +;; +;; http://www.apache.org/licenses/LICENSE-2.0 +;; +;; Unless required by applicable law or agreed to in writing, software +;; distributed under the License is distributed on an "AS IS" BASIS, +;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +;; See the License for the specific language governing permissions and +;; limitations under the License. +(ns storm.starter.clj.word-count + (:import [backtype.storm StormSubmitter LocalCluster]) + (:use [backtype.storm clojure config]) + (:gen-class)) + +(defspout sentence-spout ["sentence"] + [conf context collector] + (let [sentences ["a little brown dog" + "the man petted the dog" + "four score and seven years ago" + "an apple a day keeps the doctor away"]] + (spout + (nextTuple [] + (Thread/sleep 100) + (emit-spout! collector [(rand-nth sentences)]) + ) + (ack [id] + ;; You only need to define this method for reliable spouts + ;; (such as one that reads off of a queue like Kestrel) + ;; This is an unreliable spout, so it does nothing here + )))) + +(defspout sentence-spout-parameterized ["word"] {:params [sentences] :prepare false} + [collector] + (Thread/sleep 500) + (emit-spout! collector [(rand-nth sentences)])) + +(defbolt split-sentence ["word"] [tuple collector] + (let [words (.split (.getString tuple 0) " ")] + (doseq [w words] + (emit-bolt! collector [w] :anchor tuple)) + (ack! collector tuple) + )) + +(defbolt word-count ["word" "count"] {:prepare true} + [conf context collector] + (let [counts (atom {})] + (bolt + (execute [tuple] + (let [word (.getString tuple 0)] + (swap! counts (partial merge-with +) {word 1}) + (emit-bolt! collector [word (@counts word)] :anchor tuple) + (ack! collector tuple) + ))))) + +(defn mk-topology [] + + (topology + {"1" (spout-spec sentence-spout) + "2" (spout-spec (sentence-spout-parameterized + ["the cat jumped over the door" + "greetings from a faraway land"]) + :p 2)} + {"3" (bolt-spec {"1" :shuffle "2" :shuffle} + split-sentence + :p 5) + "4" (bolt-spec {"3" ["word"]} + word-count + :p 6)})) + +(defn run-local! [] + (let [cluster (LocalCluster.)] + (.submitTopology cluster "word-count" {TOPOLOGY-DEBUG true} (mk-topology)) + (Thread/sleep 10000) + (.shutdown cluster) + )) + +(defn submit-topology! [name] + (StormSubmitter/submitTopology + name + {TOPOLOGY-DEBUG true + TOPOLOGY-WORKERS 3} + (mk-topology))) + +(defn -main + ([] + (run-local!)) + ([name] + (submit-topology! name))) + diff --git a/examples/storm-starter/src/jvm/storm/starter/BasicDRPCTopology.java b/examples/storm-starter/src/jvm/storm/starter/BasicDRPCTopology.java new file mode 100644 index 000000000..b0493a784 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/BasicDRPCTopology.java @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.LocalDRPC; +import backtype.storm.StormSubmitter; +import backtype.storm.drpc.LinearDRPCTopologyBuilder; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +/** + * This topology is a basic example of doing distributed RPC on top of Storm. It implements a function that appends a + * "!" to any string you send the DRPC function. + *

+ * See https://github.com/nathanmarz/storm/wiki/Distributed-RPC for more information on doing distributed RPC on top of + * Storm. + */ +public class BasicDRPCTopology { + public static class ExclaimBolt extends BaseBasicBolt { + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + String input = tuple.getString(1); + collector.emit(new Values(tuple.getValue(0), input + "!")); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "result")); + } + + } + + public static void main(String[] args) throws Exception { + LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("exclamation"); + builder.addBolt(new ExclaimBolt(), 3); + + Config conf = new Config(); + + if (args == null || args.length == 0) { + LocalDRPC drpc = new LocalDRPC(); + LocalCluster cluster = new LocalCluster(); + + cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc)); + + for (String word : new String[]{ "hello", "goodbye" }) { + System.out.println("Result for \"" + word + "\": " + drpc.execute("exclamation", word)); + } + + cluster.shutdown(); + drpc.shutdown(); + } + else { + conf.setNumWorkers(3); + StormSubmitter.submitTopology(args[0], conf, builder.createRemoteTopology()); + } + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/ExclamationTopology.java b/examples/storm-starter/src/jvm/storm/starter/ExclamationTopology.java new file mode 100644 index 000000000..a0fb75793 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/ExclamationTopology.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.StormSubmitter; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.testing.TestWordSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; + +import java.util.Map; + +/** + * This is a basic example of a Storm topology. + */ +public class ExclamationTopology { + + public static class ExclamationBolt extends BaseRichBolt { + OutputCollector _collector; + + @Override + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _collector = collector; + } + + @Override + public void execute(Tuple tuple) { + _collector.emit(tuple, new Values(tuple.getString(0) + "!!!")); + _collector.ack(tuple); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + + + } + + public static void main(String[] args) throws Exception { + TopologyBuilder builder = new TopologyBuilder(); + + builder.setSpout("word", new TestWordSpout(), 10); + builder.setBolt("exclaim1", new ExclamationBolt(), 3).shuffleGrouping("word"); + builder.setBolt("exclaim2", new ExclamationBolt(), 2).shuffleGrouping("exclaim1"); + + Config conf = new Config(); + conf.setDebug(true); + + if (args != null && args.length > 0) { + conf.setNumWorkers(3); + + StormSubmitter.submitTopology(args[0], conf, builder.createTopology()); + } + else { + + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", conf, builder.createTopology()); + Utils.sleep(10000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/ManualDRPC.java b/examples/storm-starter/src/jvm/storm/starter/ManualDRPC.java new file mode 100644 index 000000000..fe0bae279 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/ManualDRPC.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.LocalDRPC; +import backtype.storm.drpc.DRPCSpout; +import backtype.storm.drpc.ReturnResults; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + + +public class ManualDRPC { + public static class ExclamationBolt extends BaseBasicBolt { + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("result", "return-info")); + } + + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + String arg = tuple.getString(0); + Object retInfo = tuple.getValue(1); + collector.emit(new Values(arg + "!!!", retInfo)); + } + + } + + public static void main(String[] args) { + TopologyBuilder builder = new TopologyBuilder(); + LocalDRPC drpc = new LocalDRPC(); + + DRPCSpout spout = new DRPCSpout("exclamation", drpc); + builder.setSpout("drpc", spout); + builder.setBolt("exclaim", new ExclamationBolt(), 3).shuffleGrouping("drpc"); + builder.setBolt("return", new ReturnResults(), 3).shuffleGrouping("exclaim"); + + LocalCluster cluster = new LocalCluster(); + Config conf = new Config(); + cluster.submitTopology("exclaim", conf, builder.createTopology()); + + System.out.println(drpc.execute("exclamation", "aaa")); + System.out.println(drpc.execute("exclamation", "bbb")); + + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/PrintSampleStream.java b/examples/storm-starter/src/jvm/storm/starter/PrintSampleStream.java new file mode 100644 index 000000000..8bead91ac --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/PrintSampleStream.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/* +// to use this example, uncomment the twitter4j dependency information in the project.clj, +// uncomment storm.starter.spout.TwitterSampleSpout, and uncomment this class + +package storm.starter; + +import storm.starter.spout.TwitterSampleSpout; +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.utils.Utils; +import storm.starter.bolt.PrinterBolt; + + +public class PrintSampleStream { + public static void main(String[] args) { + String username = args[0]; + String pwd = args[1]; + TopologyBuilder builder = new TopologyBuilder(); + + builder.setSpout("spout", new TwitterSampleSpout(username, pwd)); + builder.setBolt("print", new PrinterBolt()) + .shuffleGrouping("spout"); + + + Config conf = new Config(); + + + LocalCluster cluster = new LocalCluster(); + + cluster.submitTopology("test", conf, builder.createTopology()); + + Utils.sleep(10000); + cluster.shutdown(); + } +} +*/ \ No newline at end of file diff --git a/examples/storm-starter/src/jvm/storm/starter/ReachTopology.java b/examples/storm-starter/src/jvm/storm/starter/ReachTopology.java new file mode 100644 index 000000000..de99a91f5 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/ReachTopology.java @@ -0,0 +1,196 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.LocalDRPC; +import backtype.storm.StormSubmitter; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.drpc.LinearDRPCTopologyBuilder; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.topology.base.BaseBatchBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +import java.util.*; + +/** + * This is a good example of doing complex Distributed RPC on top of Storm. This program creates a topology that can + * compute the reach for any URL on Twitter in realtime by parallelizing the whole computation. + *

+ * Reach is the number of unique people exposed to a URL on Twitter. To compute reach, you have to get all the people + * who tweeted the URL, get all the followers of all those people, unique that set of followers, and then count the + * unique set. It's an intense computation that can involve thousands of database calls and tens of millions of follower + * records. + *

+ * This Storm topology does every piece of that computation in parallel, turning what would be a computation that takes + * minutes on a single machine into one that takes just a couple seconds. + *

+ * For the purposes of demonstration, this topology replaces the use of actual DBs with in-memory hashmaps. + *

+ * See https://github.com/nathanmarz/storm/wiki/Distributed-RPC for more information on Distributed RPC. + */ +public class ReachTopology { + public static Map> TWEETERS_DB = new HashMap>() {{ + put("foo.com/blog/1", Arrays.asList("sally", "bob", "tim", "george", "nathan")); + put("engineering.twitter.com/blog/5", Arrays.asList("adam", "david", "sally", "nathan")); + put("tech.backtype.com/blog/123", Arrays.asList("tim", "mike", "john")); + }}; + + public static Map> FOLLOWERS_DB = new HashMap>() {{ + put("sally", Arrays.asList("bob", "tim", "alice", "adam", "jim", "chris", "jai")); + put("bob", Arrays.asList("sally", "nathan", "jim", "mary", "david", "vivian")); + put("tim", Arrays.asList("alex")); + put("nathan", Arrays.asList("sally", "bob", "adam", "harry", "chris", "vivian", "emily", "jordan")); + put("adam", Arrays.asList("david", "carissa")); + put("mike", Arrays.asList("john", "bob")); + put("john", Arrays.asList("alice", "nathan", "jim", "mike", "bob")); + }}; + + public static class GetTweeters extends BaseBasicBolt { + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + Object id = tuple.getValue(0); + String url = tuple.getString(1); + List tweeters = TWEETERS_DB.get(url); + if (tweeters != null) { + for (String tweeter : tweeters) { + collector.emit(new Values(id, tweeter)); + } + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "tweeter")); + } + } + + public static class GetFollowers extends BaseBasicBolt { + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + Object id = tuple.getValue(0); + String tweeter = tuple.getString(1); + List followers = FOLLOWERS_DB.get(tweeter); + if (followers != null) { + for (String follower : followers) { + collector.emit(new Values(id, follower)); + } + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "follower")); + } + } + + public static class PartialUniquer extends BaseBatchBolt { + BatchOutputCollector _collector; + Object _id; + Set _followers = new HashSet(); + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + _followers.add(tuple.getString(1)); + } + + @Override + public void finishBatch() { + _collector.emit(new Values(_id, _followers.size())); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "partial-count")); + } + } + + public static class CountAggregator extends BaseBatchBolt { + BatchOutputCollector _collector; + Object _id; + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + _count += tuple.getInteger(1); + } + + @Override + public void finishBatch() { + _collector.emit(new Values(_id, _count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "reach")); + } + } + + public static LinearDRPCTopologyBuilder construct() { + LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("reach"); + builder.addBolt(new GetTweeters(), 4); + builder.addBolt(new GetFollowers(), 12).shuffleGrouping(); + builder.addBolt(new PartialUniquer(), 6).fieldsGrouping(new Fields("id", "follower")); + builder.addBolt(new CountAggregator(), 3).fieldsGrouping(new Fields("id")); + return builder; + } + + public static void main(String[] args) throws Exception { + LinearDRPCTopologyBuilder builder = construct(); + + + Config conf = new Config(); + + if (args == null || args.length == 0) { + conf.setMaxTaskParallelism(3); + LocalDRPC drpc = new LocalDRPC(); + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("reach-drpc", conf, builder.createLocalTopology(drpc)); + + String[] urlsToTry = new String[]{ "foo.com/blog/1", "engineering.twitter.com/blog/5", "notaurl.com" }; + for (String url : urlsToTry) { + System.out.println("Reach of " + url + ": " + drpc.execute("reach", url)); + } + + cluster.shutdown(); + drpc.shutdown(); + } + else { + conf.setNumWorkers(6); + StormSubmitter.submitTopology(args[0], conf, builder.createRemoteTopology()); + } + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/RollingTopWords.java b/examples/storm-starter/src/jvm/storm/starter/RollingTopWords.java new file mode 100644 index 000000000..2630557db --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/RollingTopWords.java @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.testing.TestWordSpout; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.tuple.Fields; +import storm.starter.bolt.IntermediateRankingsBolt; +import storm.starter.bolt.RollingCountBolt; +import storm.starter.bolt.TotalRankingsBolt; +import storm.starter.util.StormRunner; + +/** + * This topology does a continuous computation of the top N words that the topology has seen in terms of cardinality. + * The top N computation is done in a completely scalable way, and a similar approach could be used to compute things + * like trending topics or trending images on Twitter. + */ +public class RollingTopWords { + + private static final int DEFAULT_RUNTIME_IN_SECONDS = 60; + private static final int TOP_N = 5; + + private final TopologyBuilder builder; + private final String topologyName; + private final Config topologyConfig; + private final int runtimeInSeconds; + + public RollingTopWords() throws InterruptedException { + builder = new TopologyBuilder(); + topologyName = "slidingWindowCounts"; + topologyConfig = createTopologyConfiguration(); + runtimeInSeconds = DEFAULT_RUNTIME_IN_SECONDS; + + wireTopology(); + } + + private static Config createTopologyConfiguration() { + Config conf = new Config(); + conf.setDebug(true); + return conf; + } + + private void wireTopology() throws InterruptedException { + String spoutId = "wordGenerator"; + String counterId = "counter"; + String intermediateRankerId = "intermediateRanker"; + String totalRankerId = "finalRanker"; + builder.setSpout(spoutId, new TestWordSpout(), 5); + builder.setBolt(counterId, new RollingCountBolt(9, 3), 4).fieldsGrouping(spoutId, new Fields("word")); + builder.setBolt(intermediateRankerId, new IntermediateRankingsBolt(TOP_N), 4).fieldsGrouping(counterId, new Fields( + "obj")); + builder.setBolt(totalRankerId, new TotalRankingsBolt(TOP_N)).globalGrouping(intermediateRankerId); + } + + public void run() throws InterruptedException { + StormRunner.runTopologyLocally(builder.createTopology(), topologyName, topologyConfig, runtimeInSeconds); + } + + public static void main(String[] args) throws Exception { + new RollingTopWords().run(); + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/SingleJoinExample.java b/examples/storm-starter/src/jvm/storm/starter/SingleJoinExample.java new file mode 100644 index 000000000..cb1d98c3a --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/SingleJoinExample.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.testing.FeederSpout; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; +import storm.starter.bolt.SingleJoinBolt; + +public class SingleJoinExample { + public static void main(String[] args) { + FeederSpout genderSpout = new FeederSpout(new Fields("id", "gender")); + FeederSpout ageSpout = new FeederSpout(new Fields("id", "age")); + + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout("gender", genderSpout); + builder.setSpout("age", ageSpout); + builder.setBolt("join", new SingleJoinBolt(new Fields("gender", "age"))).fieldsGrouping("gender", new Fields("id")) + .fieldsGrouping("age", new Fields("id")); + + Config conf = new Config(); + conf.setDebug(true); + + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("join-example", conf, builder.createTopology()); + + for (int i = 0; i < 10; i++) { + String gender; + if (i % 2 == 0) { + gender = "male"; + } + else { + gender = "female"; + } + genderSpout.feed(new Values(i, gender)); + } + + for (int i = 9; i >= 0; i--) { + ageSpout.feed(new Values(i, i + 20)); + } + + Utils.sleep(2000); + cluster.shutdown(); + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/TransactionalGlobalCount.java b/examples/storm-starter/src/jvm/storm/starter/TransactionalGlobalCount.java new file mode 100644 index 000000000..d8ff78dff --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/TransactionalGlobalCount.java @@ -0,0 +1,173 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.testing.MemoryTransactionalSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBatchBolt; +import backtype.storm.topology.base.BaseTransactionalBolt; +import backtype.storm.transactional.ICommitter; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.transactional.TransactionalTopologyBuilder; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This is a basic example of a transactional topology. It keeps a count of the number of tuples seen so far in a + * database. The source of data and the databases are mocked out as in memory maps for demonstration purposes. This + * class is defined in depth on the wiki at https://github.com/nathanmarz/storm/wiki/Transactional-topologies + */ +public class TransactionalGlobalCount { + public static final int PARTITION_TAKE_PER_BATCH = 3; + public static final Map>> DATA = new HashMap>>() {{ + put(0, new ArrayList>() {{ + add(new Values("cat")); + add(new Values("dog")); + add(new Values("chicken")); + add(new Values("cat")); + add(new Values("dog")); + add(new Values("apple")); + }}); + put(1, new ArrayList>() {{ + add(new Values("cat")); + add(new Values("dog")); + add(new Values("apple")); + add(new Values("banana")); + }}); + put(2, new ArrayList>() {{ + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("dog")); + add(new Values("dog")); + add(new Values("dog")); + add(new Values("dog")); + }}); + }}; + + public static class Value { + int count = 0; + BigInteger txid; + } + + public static Map DATABASE = new HashMap(); + public static final String GLOBAL_COUNT_KEY = "GLOBAL-COUNT"; + + public static class BatchCount extends BaseBatchBolt { + Object _id; + BatchOutputCollector _collector; + + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + _count++; + } + + @Override + public void finishBatch() { + _collector.emit(new Values(_id, _count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "count")); + } + } + + public static class UpdateGlobalCount extends BaseTransactionalBolt implements ICommitter { + TransactionAttempt _attempt; + BatchOutputCollector _collector; + + int _sum = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt attempt) { + _collector = collector; + _attempt = attempt; + } + + @Override + public void execute(Tuple tuple) { + _sum += tuple.getInteger(1); + } + + @Override + public void finishBatch() { + Value val = DATABASE.get(GLOBAL_COUNT_KEY); + Value newval; + if (val == null || !val.txid.equals(_attempt.getTransactionId())) { + newval = new Value(); + newval.txid = _attempt.getTransactionId(); + if (val == null) { + newval.count = _sum; + } + else { + newval.count = _sum + val.count; + } + DATABASE.put(GLOBAL_COUNT_KEY, newval); + } + else { + newval = val; + } + _collector.emit(new Values(_attempt, newval.count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "sum")); + } + } + + public static void main(String[] args) throws Exception { + MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH); + TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder("global-count", "spout", spout, 3); + builder.setBolt("partial-count", new BatchCount(), 5).noneGrouping("spout"); + builder.setBolt("sum", new UpdateGlobalCount()).globalGrouping("partial-count"); + + LocalCluster cluster = new LocalCluster(); + + Config config = new Config(); + config.setDebug(true); + config.setMaxSpoutPending(3); + + cluster.submitTopology("global-count-topology", config, builder.buildTopology()); + + Thread.sleep(3000); + cluster.shutdown(); + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/TransactionalWords.java b/examples/storm-starter/src/jvm/storm/starter/TransactionalWords.java new file mode 100644 index 000000000..4d5ba1b17 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/TransactionalWords.java @@ -0,0 +1,246 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.testing.MemoryTransactionalSpout; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.topology.base.BaseTransactionalBolt; +import backtype.storm.transactional.ICommitter; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.transactional.TransactionalTopologyBuilder; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This class defines a more involved transactional topology then TransactionalGlobalCount. This topology processes a + * stream of words and produces two outputs: + *

+ * 1. A count for each word (stored in a database) 2. The number of words for every bucket of 10 counts. So it stores in + * the database how many words have appeared 0-9 times, how many have appeared 10-19 times, and so on. + *

+ * A batch of words can cause the bucket counts to decrement for some buckets and increment for others as words move + * between buckets as their counts accumulate. + */ +public class TransactionalWords { + public static class CountValue { + Integer prev_count = null; + int count = 0; + BigInteger txid = null; + } + + public static class BucketValue { + int count = 0; + BigInteger txid; + } + + public static final int BUCKET_SIZE = 10; + + public static Map COUNT_DATABASE = new HashMap(); + public static Map BUCKET_DATABASE = new HashMap(); + + + public static final int PARTITION_TAKE_PER_BATCH = 3; + + public static final Map>> DATA = new HashMap>>() {{ + put(0, new ArrayList>() {{ + add(new Values("cat")); + add(new Values("dog")); + add(new Values("chicken")); + add(new Values("cat")); + add(new Values("dog")); + add(new Values("apple")); + }}); + put(1, new ArrayList>() {{ + add(new Values("cat")); + add(new Values("dog")); + add(new Values("apple")); + add(new Values("banana")); + }}); + put(2, new ArrayList>() {{ + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("dog")); + add(new Values("dog")); + add(new Values("dog")); + add(new Values("dog")); + }}); + }}; + + public static class KeyedCountUpdater extends BaseTransactionalBolt implements ICommitter { + Map _counts = new HashMap(); + BatchOutputCollector _collector; + TransactionAttempt _id; + + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + String key = tuple.getString(1); + Integer curr = _counts.get(key); + if (curr == null) + curr = 0; + _counts.put(key, curr + 1); + } + + @Override + public void finishBatch() { + for (String key : _counts.keySet()) { + CountValue val = COUNT_DATABASE.get(key); + CountValue newVal; + if (val == null || !val.txid.equals(_id)) { + newVal = new CountValue(); + newVal.txid = _id.getTransactionId(); + if (val != null) { + newVal.prev_count = val.count; + newVal.count = val.count; + } + newVal.count = newVal.count + _counts.get(key); + COUNT_DATABASE.put(key, newVal); + } + else { + newVal = val; + } + _collector.emit(new Values(_id, key, newVal.count, newVal.prev_count)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "key", "count", "prev-count")); + } + } + + public static class Bucketize extends BaseBasicBolt { + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0); + int curr = tuple.getInteger(2); + Integer prev = tuple.getInteger(3); + + int currBucket = curr / BUCKET_SIZE; + Integer prevBucket = null; + if (prev != null) { + prevBucket = prev / BUCKET_SIZE; + } + + if (prevBucket == null) { + collector.emit(new Values(attempt, currBucket, 1)); + } + else if (currBucket != prevBucket) { + collector.emit(new Values(attempt, currBucket, 1)); + collector.emit(new Values(attempt, prevBucket, -1)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("attempt", "bucket", "delta")); + } + } + + public static class BucketCountUpdater extends BaseTransactionalBolt { + Map _accum = new HashMap(); + BatchOutputCollector _collector; + TransactionAttempt _attempt; + + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt attempt) { + _collector = collector; + _attempt = attempt; + } + + @Override + public void execute(Tuple tuple) { + Integer bucket = tuple.getInteger(1); + Integer delta = tuple.getInteger(2); + Integer curr = _accum.get(bucket); + if (curr == null) + curr = 0; + _accum.put(bucket, curr + delta); + } + + @Override + public void finishBatch() { + for (Integer bucket : _accum.keySet()) { + BucketValue currVal = BUCKET_DATABASE.get(bucket); + BucketValue newVal; + if (currVal == null || !currVal.txid.equals(_attempt.getTransactionId())) { + newVal = new BucketValue(); + newVal.txid = _attempt.getTransactionId(); + newVal.count = _accum.get(bucket); + if (currVal != null) + newVal.count += currVal.count; + BUCKET_DATABASE.put(bucket, newVal); + } + else { + newVal = currVal; + } + _collector.emit(new Values(_attempt, bucket, newVal.count)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "bucket", "count")); + } + } + + public static void main(String[] args) throws Exception { + MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH); + TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder("top-n-words", "spout", spout, 2); + builder.setBolt("count", new KeyedCountUpdater(), 5).fieldsGrouping("spout", new Fields("word")); + builder.setBolt("bucketize", new Bucketize()).noneGrouping("count"); + builder.setBolt("buckets", new BucketCountUpdater(), 5).fieldsGrouping("bucketize", new Fields("bucket")); + + + LocalCluster cluster = new LocalCluster(); + + Config config = new Config(); + config.setDebug(true); + config.setMaxSpoutPending(3); + + cluster.submitTopology("top-n-topology", config, builder.buildTopology()); + + Thread.sleep(3000); + cluster.shutdown(); + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/WordCountTopology.java b/examples/storm-starter/src/jvm/storm/starter/WordCountTopology.java new file mode 100644 index 000000000..b8dece012 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/WordCountTopology.java @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.StormSubmitter; +import backtype.storm.task.ShellBolt; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import storm.starter.spout.RandomSentenceSpout; + +import java.util.HashMap; +import java.util.Map; + +/** + * This topology demonstrates Storm's stream groupings and multilang capabilities. + */ +public class WordCountTopology { + public static class SplitSentence extends ShellBolt implements IRichBolt { + + public SplitSentence() { + super("python", "splitsentence.py"); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + + @Override + public Map getComponentConfiguration() { + return null; + } + } + + public static class WordCount extends BaseBasicBolt { + Map counts = new HashMap(); + + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + String word = tuple.getString(0); + Integer count = counts.get(word); + if (count == null) + count = 0; + count++; + counts.put(word, count); + collector.emit(new Values(word, count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word", "count")); + } + } + + public static void main(String[] args) throws Exception { + + TopologyBuilder builder = new TopologyBuilder(); + + builder.setSpout("spout", new RandomSentenceSpout(), 5); + + builder.setBolt("split", new SplitSentence(), 8).shuffleGrouping("spout"); + builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word")); + + Config conf = new Config(); + conf.setDebug(true); + + + if (args != null && args.length > 0) { + conf.setNumWorkers(3); + + StormSubmitter.submitTopology(args[0], conf, builder.createTopology()); + } + else { + conf.setMaxTaskParallelism(3); + + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("word-count", conf, builder.createTopology()); + + Thread.sleep(10000); + + cluster.shutdown(); + } + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/bolt/AbstractRankerBolt.java b/examples/storm-starter/src/jvm/storm/starter/bolt/AbstractRankerBolt.java new file mode 100644 index 000000000..cc5c0e772 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/bolt/AbstractRankerBolt.java @@ -0,0 +1,110 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.Config; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.apache.log4j.Logger; +import storm.starter.tools.Rankings; +import storm.starter.util.TupleHelpers; + +import java.util.HashMap; +import java.util.Map; + +/** + * This abstract bolt provides the basic behavior of bolts that rank objects according to their count. + *

+ * It uses a template method design pattern for {@link AbstractRankerBolt#execute(Tuple, BasicOutputCollector)} to allow + * actual bolt implementations to specify how incoming tuples are processed, i.e. how the objects embedded within those + * tuples are retrieved and counted. + */ +public abstract class AbstractRankerBolt extends BaseBasicBolt { + + private static final long serialVersionUID = 4931640198501530202L; + private static final int DEFAULT_EMIT_FREQUENCY_IN_SECONDS = 2; + private static final int DEFAULT_COUNT = 10; + + private final int emitFrequencyInSeconds; + private final int count; + private final Rankings rankings; + + public AbstractRankerBolt() { + this(DEFAULT_COUNT, DEFAULT_EMIT_FREQUENCY_IN_SECONDS); + } + + public AbstractRankerBolt(int topN) { + this(topN, DEFAULT_EMIT_FREQUENCY_IN_SECONDS); + } + + public AbstractRankerBolt(int topN, int emitFrequencyInSeconds) { + if (topN < 1) { + throw new IllegalArgumentException("topN must be >= 1 (you requested " + topN + ")"); + } + if (emitFrequencyInSeconds < 1) { + throw new IllegalArgumentException( + "The emit frequency must be >= 1 seconds (you requested " + emitFrequencyInSeconds + " seconds)"); + } + count = topN; + this.emitFrequencyInSeconds = emitFrequencyInSeconds; + rankings = new Rankings(count); + } + + protected Rankings getRankings() { + return rankings; + } + + /** + * This method functions as a template method (design pattern). + */ + @Override + public final void execute(Tuple tuple, BasicOutputCollector collector) { + if (TupleHelpers.isTickTuple(tuple)) { + getLogger().debug("Received tick tuple, triggering emit of current rankings"); + emitRankings(collector); + } + else { + updateRankingsWithTuple(tuple); + } + } + + abstract void updateRankingsWithTuple(Tuple tuple); + + private void emitRankings(BasicOutputCollector collector) { + collector.emit(new Values(rankings.copy())); + getLogger().debug("Rankings: " + rankings); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("rankings")); + } + + @Override + public Map getComponentConfiguration() { + Map conf = new HashMap(); + conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, emitFrequencyInSeconds); + return conf; + } + + abstract Logger getLogger(); +} diff --git a/examples/storm-starter/src/jvm/storm/starter/bolt/IntermediateRankingsBolt.java b/examples/storm-starter/src/jvm/storm/starter/bolt/IntermediateRankingsBolt.java new file mode 100644 index 000000000..d1805ff37 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/bolt/IntermediateRankingsBolt.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.tuple.Tuple; +import org.apache.log4j.Logger; +import storm.starter.tools.Rankable; +import storm.starter.tools.RankableObjectWithFields; + +/** + * This bolt ranks incoming objects by their count. + *

+ * It assumes the input tuples to adhere to the following format: (object, object_count, additionalField1, + * additionalField2, ..., additionalFieldN). + */ +public final class IntermediateRankingsBolt extends AbstractRankerBolt { + + private static final long serialVersionUID = -1369800530256637409L; + private static final Logger LOG = Logger.getLogger(IntermediateRankingsBolt.class); + + public IntermediateRankingsBolt() { + super(); + } + + public IntermediateRankingsBolt(int topN) { + super(topN); + } + + public IntermediateRankingsBolt(int topN, int emitFrequencyInSeconds) { + super(topN, emitFrequencyInSeconds); + } + + @Override + void updateRankingsWithTuple(Tuple tuple) { + Rankable rankable = RankableObjectWithFields.from(tuple); + super.getRankings().updateWith(rankable); + } + + @Override + Logger getLogger() { + return LOG; + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/bolt/PrinterBolt.java b/examples/storm-starter/src/jvm/storm/starter/bolt/PrinterBolt.java new file mode 100644 index 000000000..58fc8caf1 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/bolt/PrinterBolt.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Tuple; + + +public class PrinterBolt extends BaseBasicBolt { + + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + System.out.println(tuple); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer ofd) { + } + +} diff --git a/examples/storm-starter/src/jvm/storm/starter/bolt/RollingCountBolt.java b/examples/storm-starter/src/jvm/storm/starter/bolt/RollingCountBolt.java new file mode 100644 index 000000000..f83906cea --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/bolt/RollingCountBolt.java @@ -0,0 +1,142 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.Config; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.apache.log4j.Logger; +import storm.starter.tools.NthLastModifiedTimeTracker; +import storm.starter.tools.SlidingWindowCounter; +import storm.starter.util.TupleHelpers; + +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +/** + * This bolt performs rolling counts of incoming objects, i.e. sliding window based counting. + *

+ * The bolt is configured by two parameters, the length of the sliding window in seconds (which influences the output + * data of the bolt, i.e. how it will count objects) and the emit frequency in seconds (which influences how often the + * bolt will output the latest window counts). For instance, if the window length is set to an equivalent of five + * minutes and the emit frequency to one minute, then the bolt will output the latest five-minute sliding window every + * minute. + *

+ * The bolt emits a rolling count tuple per object, consisting of the object itself, its latest rolling count, and the + * actual duration of the sliding window. The latter is included in case the expected sliding window length (as + * configured by the user) is different from the actual length, e.g. due to high system load. Note that the actual + * window length is tracked and calculated for the window, and not individually for each object within a window. + *

+ * Note: During the startup phase you will usually observe that the bolt warns you about the actual sliding window + * length being smaller than the expected length. This behavior is expected and is caused by the way the sliding window + * counts are initially "loaded up". You can safely ignore this warning during startup (e.g. you will see this warning + * during the first ~ five minutes of startup time if the window length is set to five minutes). + */ +public class RollingCountBolt extends BaseRichBolt { + + private static final long serialVersionUID = 5537727428628598519L; + private static final Logger LOG = Logger.getLogger(RollingCountBolt.class); + private static final int NUM_WINDOW_CHUNKS = 5; + private static final int DEFAULT_SLIDING_WINDOW_IN_SECONDS = NUM_WINDOW_CHUNKS * 60; + private static final int DEFAULT_EMIT_FREQUENCY_IN_SECONDS = DEFAULT_SLIDING_WINDOW_IN_SECONDS / NUM_WINDOW_CHUNKS; + private static final String WINDOW_LENGTH_WARNING_TEMPLATE = + "Actual window length is %d seconds when it should be %d seconds" + + " (you can safely ignore this warning during the startup phase)"; + + private final SlidingWindowCounter counter; + private final int windowLengthInSeconds; + private final int emitFrequencyInSeconds; + private OutputCollector collector; + private NthLastModifiedTimeTracker lastModifiedTracker; + + public RollingCountBolt() { + this(DEFAULT_SLIDING_WINDOW_IN_SECONDS, DEFAULT_EMIT_FREQUENCY_IN_SECONDS); + } + + public RollingCountBolt(int windowLengthInSeconds, int emitFrequencyInSeconds) { + this.windowLengthInSeconds = windowLengthInSeconds; + this.emitFrequencyInSeconds = emitFrequencyInSeconds; + counter = new SlidingWindowCounter(deriveNumWindowChunksFrom(this.windowLengthInSeconds, + this.emitFrequencyInSeconds)); + } + + private int deriveNumWindowChunksFrom(int windowLengthInSeconds, int windowUpdateFrequencyInSeconds) { + return windowLengthInSeconds / windowUpdateFrequencyInSeconds; + } + + @SuppressWarnings("rawtypes") + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + this.collector = collector; + lastModifiedTracker = new NthLastModifiedTimeTracker(deriveNumWindowChunksFrom(this.windowLengthInSeconds, + this.emitFrequencyInSeconds)); + } + + @Override + public void execute(Tuple tuple) { + if (TupleHelpers.isTickTuple(tuple)) { + LOG.debug("Received tick tuple, triggering emit of current window counts"); + emitCurrentWindowCounts(); + } + else { + countObjAndAck(tuple); + } + } + + private void emitCurrentWindowCounts() { + Map counts = counter.getCountsThenAdvanceWindow(); + int actualWindowLengthInSeconds = lastModifiedTracker.secondsSinceOldestModification(); + lastModifiedTracker.markAsModified(); + if (actualWindowLengthInSeconds != windowLengthInSeconds) { + LOG.warn(String.format(WINDOW_LENGTH_WARNING_TEMPLATE, actualWindowLengthInSeconds, windowLengthInSeconds)); + } + emit(counts, actualWindowLengthInSeconds); + } + + private void emit(Map counts, int actualWindowLengthInSeconds) { + for (Entry entry : counts.entrySet()) { + Object obj = entry.getKey(); + Long count = entry.getValue(); + collector.emit(new Values(obj, count, actualWindowLengthInSeconds)); + } + } + + private void countObjAndAck(Tuple tuple) { + Object obj = tuple.getValue(0); + counter.incrementCount(obj); + collector.ack(tuple); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("obj", "count", "actualWindowLengthInSeconds")); + } + + @Override + public Map getComponentConfiguration() { + Map conf = new HashMap(); + conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, emitFrequencyInSeconds); + return conf; + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/bolt/SingleJoinBolt.java b/examples/storm-starter/src/jvm/storm/starter/bolt/SingleJoinBolt.java new file mode 100644 index 000000000..85a7a267e --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/bolt/SingleJoinBolt.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.Config; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.TimeCacheMap; + +import java.util.*; + +public class SingleJoinBolt extends BaseRichBolt { + OutputCollector _collector; + Fields _idFields; + Fields _outFields; + int _numSources; + TimeCacheMap, Map> _pending; + Map _fieldLocations; + + public SingleJoinBolt(Fields outFields) { + _outFields = outFields; + } + + @Override + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _fieldLocations = new HashMap(); + _collector = collector; + int timeout = ((Number) conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)).intValue(); + _pending = new TimeCacheMap, Map>(timeout, new ExpireCallback()); + _numSources = context.getThisSources().size(); + Set idFields = null; + for (GlobalStreamId source : context.getThisSources().keySet()) { + Fields fields = context.getComponentOutputFields(source.get_componentId(), source.get_streamId()); + Set setFields = new HashSet(fields.toList()); + if (idFields == null) + idFields = setFields; + else + idFields.retainAll(setFields); + + for (String outfield : _outFields) { + for (String sourcefield : fields) { + if (outfield.equals(sourcefield)) { + _fieldLocations.put(outfield, source); + } + } + } + } + _idFields = new Fields(new ArrayList(idFields)); + + if (_fieldLocations.size() != _outFields.size()) { + throw new RuntimeException("Cannot find all outfields among sources"); + } + } + + @Override + public void execute(Tuple tuple) { + List id = tuple.select(_idFields); + GlobalStreamId streamId = new GlobalStreamId(tuple.getSourceComponent(), tuple.getSourceStreamId()); + if (!_pending.containsKey(id)) { + _pending.put(id, new HashMap()); + } + Map parts = _pending.get(id); + if (parts.containsKey(streamId)) + throw new RuntimeException("Received same side of single join twice"); + parts.put(streamId, tuple); + if (parts.size() == _numSources) { + _pending.remove(id); + List joinResult = new ArrayList(); + for (String outField : _outFields) { + GlobalStreamId loc = _fieldLocations.get(outField); + joinResult.add(parts.get(loc).getValueByField(outField)); + } + _collector.emit(new ArrayList(parts.values()), joinResult); + + for (Tuple part : parts.values()) { + _collector.ack(part); + } + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(_outFields); + } + + private class ExpireCallback implements TimeCacheMap.ExpiredCallback, Map> { + @Override + public void expire(List id, Map tuples) { + for (Tuple tuple : tuples.values()) { + _collector.fail(tuple); + } + } + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/bolt/TotalRankingsBolt.java b/examples/storm-starter/src/jvm/storm/starter/bolt/TotalRankingsBolt.java new file mode 100644 index 000000000..0e1bb05a7 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/bolt/TotalRankingsBolt.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.tuple.Tuple; +import org.apache.log4j.Logger; +import storm.starter.tools.Rankings; + +/** + * This bolt merges incoming {@link Rankings}. + *

+ * It can be used to merge intermediate rankings generated by {@link IntermediateRankingsBolt} into a final, + * consolidated ranking. To do so, configure this bolt with a globalGrouping on {@link IntermediateRankingsBolt}. + */ +public final class TotalRankingsBolt extends AbstractRankerBolt { + + private static final long serialVersionUID = -8447525895532302198L; + private static final Logger LOG = Logger.getLogger(TotalRankingsBolt.class); + + public TotalRankingsBolt() { + super(); + } + + public TotalRankingsBolt(int topN) { + super(topN); + } + + public TotalRankingsBolt(int topN, int emitFrequencyInSeconds) { + super(topN, emitFrequencyInSeconds); + } + + @Override + void updateRankingsWithTuple(Tuple tuple) { + Rankings rankingsToBeMerged = (Rankings) tuple.getValue(0); + super.getRankings().updateWith(rankingsToBeMerged); + super.getRankings().pruneZeroCounts(); + } + + @Override + Logger getLogger() { + return LOG; + } + +} diff --git a/examples/storm-starter/src/jvm/storm/starter/spout/RandomSentenceSpout.java b/examples/storm-starter/src/jvm/storm/starter/spout/RandomSentenceSpout.java new file mode 100644 index 000000000..813b10cf3 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/spout/RandomSentenceSpout.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.spout; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; + +import java.util.Map; +import java.util.Random; + +public class RandomSentenceSpout extends BaseRichSpout { + SpoutOutputCollector _collector; + Random _rand; + + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _collector = collector; + _rand = new Random(); + } + + @Override + public void nextTuple() { + Utils.sleep(100); + String[] sentences = new String[]{ "the cow jumped over the moon", "an apple a day keeps the doctor away", + "four score and seven years ago", "snow white and the seven dwarfs", "i am at two with nature" }; + String sentence = sentences[_rand.nextInt(sentences.length)]; + _collector.emit(new Values(sentence)); + } + + @Override + public void ack(Object id) { + } + + @Override + public void fail(Object id) { + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + +} \ No newline at end of file diff --git a/examples/storm-starter/src/jvm/storm/starter/spout/TwitterSampleSpout.java b/examples/storm-starter/src/jvm/storm/starter/spout/TwitterSampleSpout.java new file mode 100644 index 000000000..5cc12de16 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/spout/TwitterSampleSpout.java @@ -0,0 +1,122 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/* + +package storm.starter.spout; + +import backtype.storm.Config; +import twitter4j.conf.ConfigurationBuilder; +import twitter4j.TwitterStream; +import twitter4j.TwitterStreamFactory; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; +import java.util.Map; +import java.util.concurrent.LinkedBlockingQueue; +import twitter4j.Status; +import twitter4j.StatusDeletionNotice; +import twitter4j.StatusListener; + +public class TwitterSampleSpout extends BaseRichSpout { + SpoutOutputCollector _collector; + LinkedBlockingQueue queue = null; + TwitterStream _twitterStream; + String _username; + String _pwd; + + + public TwitterSampleSpout(String username, String pwd) { + _username = username; + _pwd = pwd; + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + queue = new LinkedBlockingQueue(1000); + _collector = collector; + StatusListener listener = new StatusListener() { + + @Override + public void onStatus(Status status) { + queue.offer(status); + } + + @Override + public void onDeletionNotice(StatusDeletionNotice sdn) { + } + + @Override + public void onTrackLimitationNotice(int i) { + } + + @Override + public void onScrubGeo(long l, long l1) { + } + + @Override + public void onException(Exception e) { + } + + }; + TwitterStreamFactory fact = new TwitterStreamFactory(new ConfigurationBuilder().setUser(_username).setPassword(_pwd).build()); + _twitterStream = fact.getInstance(); + _twitterStream.addListener(listener); + _twitterStream.sample(); + } + + @Override + public void nextTuple() { + Status ret = queue.poll(); + if(ret==null) { + Utils.sleep(50); + } else { + _collector.emit(new Values(ret)); + } + } + + @Override + public void close() { + _twitterStream.shutdown(); + } + + @Override + public Map getComponentConfiguration() { + Config ret = new Config(); + ret.setMaxTaskParallelism(1); + return ret; + } + + @Override + public void ack(Object id) { + } + + @Override + public void fail(Object id) { + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("tweet")); + } + +} +*/ \ No newline at end of file diff --git a/examples/storm-starter/src/jvm/storm/starter/tools/NthLastModifiedTimeTracker.java b/examples/storm-starter/src/jvm/storm/starter/tools/NthLastModifiedTimeTracker.java new file mode 100644 index 000000000..08df8cfd0 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/tools/NthLastModifiedTimeTracker.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import backtype.storm.utils.Time; +import org.apache.commons.collections.buffer.CircularFifoBuffer; + +/** + * This class tracks the time-since-last-modify of a "thing" in a rolling fashion. + *

+ * For example, create a 5-slot tracker to track the five most recent time-since-last-modify. + *

+ * You must manually "mark" that the "something" that you want to track -- in terms of modification times -- has just + * been modified. + */ +public class NthLastModifiedTimeTracker { + + private static final int MILLIS_IN_SEC = 1000; + + private final CircularFifoBuffer lastModifiedTimesMillis; + + public NthLastModifiedTimeTracker(int numTimesToTrack) { + if (numTimesToTrack < 1) { + throw new IllegalArgumentException( + "numTimesToTrack must be greater than zero (you requested " + numTimesToTrack + ")"); + } + lastModifiedTimesMillis = new CircularFifoBuffer(numTimesToTrack); + initLastModifiedTimesMillis(); + } + + private void initLastModifiedTimesMillis() { + long nowCached = now(); + for (int i = 0; i < lastModifiedTimesMillis.maxSize(); i++) { + lastModifiedTimesMillis.add(Long.valueOf(nowCached)); + } + } + + private long now() { + return Time.currentTimeMillis(); + } + + public int secondsSinceOldestModification() { + long modifiedTimeMillis = ((Long) lastModifiedTimesMillis.get()).longValue(); + return (int) ((now() - modifiedTimeMillis) / MILLIS_IN_SEC); + } + + public void markAsModified() { + updateLastModifiedTime(); + } + + private void updateLastModifiedTime() { + lastModifiedTimesMillis.add(now()); + } + +} diff --git a/examples/storm-starter/src/jvm/storm/starter/tools/Rankable.java b/examples/storm-starter/src/jvm/storm/starter/tools/Rankable.java new file mode 100644 index 000000000..85e3d1d0f --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/tools/Rankable.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +public interface Rankable extends Comparable { + + Object getObject(); + + long getCount(); + + /** + * Note: We do not defensively copy the object wrapped by the Rankable. It is passed as is. + * + * @return a defensive copy + */ + Rankable copy(); +} diff --git a/examples/storm-starter/src/jvm/storm/starter/tools/RankableObjectWithFields.java b/examples/storm-starter/src/jvm/storm/starter/tools/RankableObjectWithFields.java new file mode 100644 index 000000000..9a0ecae6a --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/tools/RankableObjectWithFields.java @@ -0,0 +1,148 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import backtype.storm.tuple.Tuple; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.io.Serializable; +import java.util.List; + +/** + * This class wraps an objects and its associated count, including any additional data fields. + *

+ * This class can be used, for instance, to track the number of occurrences of an object in a Storm topology. + */ +public class RankableObjectWithFields implements Rankable, Serializable { + + private static final long serialVersionUID = -9102878650001058090L; + private static final String toStringSeparator = "|"; + + private final Object obj; + private final long count; + private final ImmutableList fields; + + public RankableObjectWithFields(Object obj, long count, Object... otherFields) { + if (obj == null) { + throw new IllegalArgumentException("The object must not be null"); + } + if (count < 0) { + throw new IllegalArgumentException("The count must be >= 0"); + } + this.obj = obj; + this.count = count; + fields = ImmutableList.copyOf(otherFields); + + } + + /** + * Construct a new instance based on the provided {@link Tuple}. + *

+ * This method expects the object to be ranked in the first field (index 0) of the provided tuple, and the number of + * occurrences of the object (its count) in the second field (index 1). Any further fields in the tuple will be + * extracted and tracked, too. These fields can be accessed via {@link RankableObjectWithFields#getFields()}. + * + * @param tuple + * + * @return new instance based on the provided tuple + */ + public static RankableObjectWithFields from(Tuple tuple) { + List otherFields = Lists.newArrayList(tuple.getValues()); + Object obj = otherFields.remove(0); + Long count = (Long) otherFields.remove(0); + return new RankableObjectWithFields(obj, count, otherFields.toArray()); + } + + public Object getObject() { + return obj; + } + + public long getCount() { + return count; + } + + /** + * @return an immutable list of any additional data fields of the object (may be empty but will never be null) + */ + public List getFields() { + return fields; + } + + @Override + public int compareTo(Rankable other) { + long delta = this.getCount() - other.getCount(); + if (delta > 0) { + return 1; + } + else if (delta < 0) { + return -1; + } + else { + return 0; + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof RankableObjectWithFields)) { + return false; + } + RankableObjectWithFields other = (RankableObjectWithFields) o; + return obj.equals(other.obj) && count == other.count; + } + + @Override + public int hashCode() { + int result = 17; + int countHash = (int) (count ^ (count >>> 32)); + result = 31 * result + countHash; + result = 31 * result + obj.hashCode(); + return result; + } + + public String toString() { + StringBuffer buf = new StringBuffer(); + buf.append("["); + buf.append(obj); + buf.append(toStringSeparator); + buf.append(count); + for (Object field : fields) { + buf.append(toStringSeparator); + buf.append(field); + } + buf.append("]"); + return buf.toString(); + } + + /** + * Note: We do not defensively copy the wrapped object and any accompanying fields. We do guarantee, however, + * do return a defensive (shallow) copy of the List object that is wrapping any accompanying fields. + * + * @return + */ + @Override + public Rankable copy() { + List shallowCopyOfFields = ImmutableList.copyOf(getFields()); + return new RankableObjectWithFields(getObject(), getCount(), shallowCopyOfFields); + } + +} diff --git a/examples/storm-starter/src/jvm/storm/starter/tools/Rankings.java b/examples/storm-starter/src/jvm/storm/starter/tools/Rankings.java new file mode 100644 index 000000000..551ebfbe3 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/tools/Rankings.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; + +public class Rankings implements Serializable { + + private static final long serialVersionUID = -1549827195410578903L; + private static final int DEFAULT_COUNT = 10; + + private final int maxSize; + private final List rankedItems = Lists.newArrayList(); + + public Rankings() { + this(DEFAULT_COUNT); + } + + public Rankings(int topN) { + if (topN < 1) { + throw new IllegalArgumentException("topN must be >= 1"); + } + maxSize = topN; + } + + /** + * Copy constructor. + * @param other + */ + public Rankings(Rankings other) { + this(other.maxSize()); + updateWith(other); + } + + /** + * @return the maximum possible number (size) of ranked objects this instance can hold + */ + public int maxSize() { + return maxSize; + } + + /** + * @return the number (size) of ranked objects this instance is currently holding + */ + public int size() { + return rankedItems.size(); + } + + /** + * The returned defensive copy is only "somewhat" defensive. We do, for instance, return a defensive copy of the + * enclosing List instance, and we do try to defensively copy any contained Rankable objects, too. However, the + * contract of {@link storm.starter.tools.Rankable#copy()} does not guarantee that any Object's embedded within + * a Rankable will be defensively copied, too. + * + * @return a somewhat defensive copy of ranked items + */ + public List getRankings() { + List copy = Lists.newLinkedList(); + for (Rankable r: rankedItems) { + copy.add(r.copy()); + } + return ImmutableList.copyOf(copy); + } + + public void updateWith(Rankings other) { + for (Rankable r : other.getRankings()) { + updateWith(r); + } + } + + public void updateWith(Rankable r) { + synchronized(rankedItems) { + addOrReplace(r); + rerank(); + shrinkRankingsIfNeeded(); + } + } + + private void addOrReplace(Rankable r) { + Integer rank = findRankOf(r); + if (rank != null) { + rankedItems.set(rank, r); + } + else { + rankedItems.add(r); + } + } + + private Integer findRankOf(Rankable r) { + Object tag = r.getObject(); + for (int rank = 0; rank < rankedItems.size(); rank++) { + Object cur = rankedItems.get(rank).getObject(); + if (cur.equals(tag)) { + return rank; + } + } + return null; + } + + private void rerank() { + Collections.sort(rankedItems); + Collections.reverse(rankedItems); + } + + private void shrinkRankingsIfNeeded() { + if (rankedItems.size() > maxSize) { + rankedItems.remove(maxSize); + } + } + + /** + * Removes ranking entries that have a count of zero. + */ + public void pruneZeroCounts() { + int i = 0; + while (i < rankedItems.size()) { + if (rankedItems.get(i).getCount() == 0) { + rankedItems.remove(i); + } + else { + i++; + } + } + } + + public String toString() { + return rankedItems.toString(); + } + + /** + * Creates a (defensive) copy of itself. + */ + public Rankings copy() { + return new Rankings(this); + } +} \ No newline at end of file diff --git a/examples/storm-starter/src/jvm/storm/starter/tools/SlidingWindowCounter.java b/examples/storm-starter/src/jvm/storm/starter/tools/SlidingWindowCounter.java new file mode 100644 index 000000000..1199c4011 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/tools/SlidingWindowCounter.java @@ -0,0 +1,119 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import java.io.Serializable; +import java.util.Map; + +/** + * This class counts objects in a sliding window fashion. + *

+ * It is designed 1) to give multiple "producer" threads write access to the counter, i.e. being able to increment + * counts of objects, and 2) to give a single "consumer" thread (e.g. {@link PeriodicSlidingWindowCounter}) read access + * to the counter. Whenever the consumer thread performs a read operation, this class will advance the head slot of the + * sliding window counter. This means that the consumer thread indirectly controls where writes of the producer threads + * will go to. Also, by itself this class will not advance the head slot. + *

+ * A note for analyzing data based on a sliding window count: During the initial windowLengthInSlots + * iterations, this sliding window counter will always return object counts that are equal or greater than in the + * previous iteration. This is the effect of the counter "loading up" at the very start of its existence. Conceptually, + * this is the desired behavior. + *

+ * To give an example, using a counter with 5 slots which for the sake of this example represent 1 minute of time each: + *

+ *

+ * {@code
+ * Sliding window counts of an object X over time
+ *
+ * Minute (timeline):
+ * 1    2   3   4   5   6   7   8
+ *
+ * Observed counts per minute:
+ * 1    1   1   1   0   0   0   0
+ *
+ * Counts returned by counter:
+ * 1    2   3   4   4   3   2   1
+ * }
+ * 
+ *

+ * As you can see in this example, for the first windowLengthInSlots (here: the first five minutes) the + * counter will always return counts equal or greater than in the previous iteration (1, 2, 3, 4, 4). This initial load + * effect needs to be accounted for whenever you want to perform analyses such as trending topics; otherwise your + * analysis algorithm might falsely identify the object to be trending as the counter seems to observe continuously + * increasing counts. Also, note that during the initial load phase every object will exhibit increasing + * counts. + *

+ * On a high-level, the counter exhibits the following behavior: If you asked the example counter after two minutes, + * "how often did you count the object during the past five minutes?", then it should reply "I have counted it 2 times + * in the past five minutes", implying that it can only account for the last two of those five minutes because the + * counter was not running before that time. + * + * @param The type of those objects we want to count. + */ +public final class SlidingWindowCounter implements Serializable { + + private static final long serialVersionUID = -2645063988768785810L; + + private SlotBasedCounter objCounter; + private int headSlot; + private int tailSlot; + private int windowLengthInSlots; + + public SlidingWindowCounter(int windowLengthInSlots) { + if (windowLengthInSlots < 2) { + throw new IllegalArgumentException( + "Window length in slots must be at least two (you requested " + windowLengthInSlots + ")"); + } + this.windowLengthInSlots = windowLengthInSlots; + this.objCounter = new SlotBasedCounter(this.windowLengthInSlots); + + this.headSlot = 0; + this.tailSlot = slotAfter(headSlot); + } + + public void incrementCount(T obj) { + objCounter.incrementCount(obj, headSlot); + } + + /** + * Return the current (total) counts of all tracked objects, then advance the window. + *

+ * Whenever this method is called, we consider the counts of the current sliding window to be available to and + * successfully processed "upstream" (i.e. by the caller). Knowing this we will start counting any subsequent + * objects within the next "chunk" of the sliding window. + * + * @return The current (total) counts of all tracked objects. + */ + public Map getCountsThenAdvanceWindow() { + Map counts = objCounter.getCounts(); + objCounter.wipeZeros(); + objCounter.wipeSlot(tailSlot); + advanceHead(); + return counts; + } + + private void advanceHead() { + headSlot = tailSlot; + tailSlot = slotAfter(tailSlot); + } + + private int slotAfter(int slot) { + return (slot + 1) % windowLengthInSlots; + } + +} diff --git a/examples/storm-starter/src/jvm/storm/starter/tools/SlotBasedCounter.java b/examples/storm-starter/src/jvm/storm/starter/tools/SlotBasedCounter.java new file mode 100644 index 000000000..4b2d472dc --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/tools/SlotBasedCounter.java @@ -0,0 +1,118 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * This class provides per-slot counts of the occurrences of objects. + *

+ * It can be used, for instance, as a building block for implementing sliding window counting of objects. + * + * @param The type of those objects we want to count. + */ +public final class SlotBasedCounter implements Serializable { + + private static final long serialVersionUID = 4858185737378394432L; + + private final Map objToCounts = new HashMap(); + private final int numSlots; + + public SlotBasedCounter(int numSlots) { + if (numSlots <= 0) { + throw new IllegalArgumentException("Number of slots must be greater than zero (you requested " + numSlots + ")"); + } + this.numSlots = numSlots; + } + + public void incrementCount(T obj, int slot) { + long[] counts = objToCounts.get(obj); + if (counts == null) { + counts = new long[this.numSlots]; + objToCounts.put(obj, counts); + } + counts[slot]++; + } + + public long getCount(T obj, int slot) { + long[] counts = objToCounts.get(obj); + if (counts == null) { + return 0; + } + else { + return counts[slot]; + } + } + + public Map getCounts() { + Map result = new HashMap(); + for (T obj : objToCounts.keySet()) { + result.put(obj, computeTotalCount(obj)); + } + return result; + } + + private long computeTotalCount(T obj) { + long[] curr = objToCounts.get(obj); + long total = 0; + for (long l : curr) { + total += l; + } + return total; + } + + /** + * Reset the slot count of any tracked objects to zero for the given slot. + * + * @param slot + */ + public void wipeSlot(int slot) { + for (T obj : objToCounts.keySet()) { + resetSlotCountToZero(obj, slot); + } + } + + private void resetSlotCountToZero(T obj, int slot) { + long[] counts = objToCounts.get(obj); + counts[slot] = 0; + } + + private boolean shouldBeRemovedFromCounter(T obj) { + return computeTotalCount(obj) == 0; + } + + /** + * Remove any object from the counter whose total count is zero (to free up memory). + */ + public void wipeZeros() { + Set objToBeRemoved = new HashSet(); + for (T obj : objToCounts.keySet()) { + if (shouldBeRemovedFromCounter(obj)) { + objToBeRemoved.add(obj); + } + } + for (T obj : objToBeRemoved) { + objToCounts.remove(obj); + } + } + +} diff --git a/examples/storm-starter/src/jvm/storm/starter/trident/TridentReach.java b/examples/storm-starter/src/jvm/storm/starter/trident/TridentReach.java new file mode 100644 index 000000000..2d87c474e --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/trident/TridentReach.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.trident; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.LocalDRPC; +import backtype.storm.generated.StormTopology; +import backtype.storm.task.IMetricsContext; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import storm.trident.TridentState; +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.CombinerAggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.builtin.MapGet; +import storm.trident.operation.builtin.Sum; +import storm.trident.state.ReadOnlyState; +import storm.trident.state.State; +import storm.trident.state.StateFactory; +import storm.trident.state.map.ReadOnlyMapState; +import storm.trident.tuple.TridentTuple; + +import java.util.*; + +public class TridentReach { + public static Map> TWEETERS_DB = new HashMap>() {{ + put("foo.com/blog/1", Arrays.asList("sally", "bob", "tim", "george", "nathan")); + put("engineering.twitter.com/blog/5", Arrays.asList("adam", "david", "sally", "nathan")); + put("tech.backtype.com/blog/123", Arrays.asList("tim", "mike", "john")); + }}; + + public static Map> FOLLOWERS_DB = new HashMap>() {{ + put("sally", Arrays.asList("bob", "tim", "alice", "adam", "jim", "chris", "jai")); + put("bob", Arrays.asList("sally", "nathan", "jim", "mary", "david", "vivian")); + put("tim", Arrays.asList("alex")); + put("nathan", Arrays.asList("sally", "bob", "adam", "harry", "chris", "vivian", "emily", "jordan")); + put("adam", Arrays.asList("david", "carissa")); + put("mike", Arrays.asList("john", "bob")); + put("john", Arrays.asList("alice", "nathan", "jim", "mike", "bob")); + }}; + + public static class StaticSingleKeyMapState extends ReadOnlyState implements ReadOnlyMapState { + public static class Factory implements StateFactory { + Map _map; + + public Factory(Map map) { + _map = map; + } + + @Override + public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) { + return new StaticSingleKeyMapState(_map); + } + + } + + Map _map; + + public StaticSingleKeyMapState(Map map) { + _map = map; + } + + + @Override + public List multiGet(List> keys) { + List ret = new ArrayList(); + for (List key : keys) { + Object singleKey = key.get(0); + ret.add(_map.get(singleKey)); + } + return ret; + } + + } + + public static class One implements CombinerAggregator { + @Override + public Integer init(TridentTuple tuple) { + return 1; + } + + @Override + public Integer combine(Integer val1, Integer val2) { + return 1; + } + + @Override + public Integer zero() { + return 1; + } + } + + public static class ExpandList extends BaseFunction { + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + List l = (List) tuple.getValue(0); + if (l != null) { + for (Object o : l) { + collector.emit(new Values(o)); + } + } + } + + } + + public static StormTopology buildTopology(LocalDRPC drpc) { + TridentTopology topology = new TridentTopology(); + TridentState urlToTweeters = topology.newStaticState(new StaticSingleKeyMapState.Factory(TWEETERS_DB)); + TridentState tweetersToFollowers = topology.newStaticState(new StaticSingleKeyMapState.Factory(FOLLOWERS_DB)); + + + topology.newDRPCStream("reach", drpc).stateQuery(urlToTweeters, new Fields("args"), new MapGet(), new Fields( + "tweeters")).each(new Fields("tweeters"), new ExpandList(), new Fields("tweeter")).shuffle().stateQuery( + tweetersToFollowers, new Fields("tweeter"), new MapGet(), new Fields("followers")).each(new Fields("followers"), + new ExpandList(), new Fields("follower")).groupBy(new Fields("follower")).aggregate(new One(), new Fields( + "one")).aggregate(new Fields("one"), new Sum(), new Fields("reach")); + return topology.build(); + } + + public static void main(String[] args) throws Exception { + LocalDRPC drpc = new LocalDRPC(); + + Config conf = new Config(); + LocalCluster cluster = new LocalCluster(); + + cluster.submitTopology("reach", conf, buildTopology(drpc)); + + Thread.sleep(2000); + + System.out.println("REACH: " + drpc.execute("reach", "aaa")); + System.out.println("REACH: " + drpc.execute("reach", "foo.com/blog/1")); + System.out.println("REACH: " + drpc.execute("reach", "engineering.twitter.com/blog/5")); + + + cluster.shutdown(); + drpc.shutdown(); + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/trident/TridentWordCount.java b/examples/storm-starter/src/jvm/storm/starter/trident/TridentWordCount.java new file mode 100644 index 000000000..439e00f48 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/trident/TridentWordCount.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.trident; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.LocalDRPC; +import backtype.storm.StormSubmitter; +import backtype.storm.generated.StormTopology; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import storm.trident.TridentState; +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.builtin.Count; +import storm.trident.operation.builtin.FilterNull; +import storm.trident.operation.builtin.MapGet; +import storm.trident.operation.builtin.Sum; +import storm.trident.testing.FixedBatchSpout; +import storm.trident.testing.MemoryMapState; +import storm.trident.tuple.TridentTuple; + + +public class TridentWordCount { + public static class Split extends BaseFunction { + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + String sentence = tuple.getString(0); + for (String word : sentence.split(" ")) { + collector.emit(new Values(word)); + } + } + } + + public static StormTopology buildTopology(LocalDRPC drpc) { + FixedBatchSpout spout = new FixedBatchSpout(new Fields("sentence"), 3, new Values("the cow jumped over the moon"), + new Values("the man went to the store and bought some candy"), new Values("four score and seven years ago"), + new Values("how many apples can you eat"), new Values("to be or not to be the person")); + spout.setCycle(true); + + TridentTopology topology = new TridentTopology(); + TridentState wordCounts = topology.newStream("spout1", spout).parallelismHint(16).each(new Fields("sentence"), + new Split(), new Fields("word")).groupBy(new Fields("word")).persistentAggregate(new MemoryMapState.Factory(), + new Count(), new Fields("count")).parallelismHint(16); + + topology.newDRPCStream("words", drpc).each(new Fields("args"), new Split(), new Fields("word")).groupBy(new Fields( + "word")).stateQuery(wordCounts, new Fields("word"), new MapGet(), new Fields("count")).each(new Fields("count"), + new FilterNull()).aggregate(new Fields("count"), new Sum(), new Fields("sum")); + return topology.build(); + } + + public static void main(String[] args) throws Exception { + Config conf = new Config(); + conf.setMaxSpoutPending(20); + if (args.length == 0) { + LocalDRPC drpc = new LocalDRPC(); + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("wordCounter", conf, buildTopology(drpc)); + for (int i = 0; i < 100; i++) { + System.out.println("DRPC RESULT: " + drpc.execute("words", "cat the dog jumped")); + Thread.sleep(1000); + } + } + else { + conf.setNumWorkers(3); + StormSubmitter.submitTopology(args[0], conf, buildTopology(null)); + } + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/util/StormRunner.java b/examples/storm-starter/src/jvm/storm/starter/util/StormRunner.java new file mode 100644 index 000000000..f916ec61b --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/util/StormRunner.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.util; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.generated.StormTopology; + +public final class StormRunner { + + private static final int MILLIS_IN_SEC = 1000; + + private StormRunner() { + } + + public static void runTopologyLocally(StormTopology topology, String topologyName, Config conf, int runtimeInSeconds) + throws InterruptedException { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology(topologyName, conf, topology); + Thread.sleep((long) runtimeInSeconds * MILLIS_IN_SEC); + cluster.killTopology(topologyName); + cluster.shutdown(); + } +} diff --git a/examples/storm-starter/src/jvm/storm/starter/util/TupleHelpers.java b/examples/storm-starter/src/jvm/storm/starter/util/TupleHelpers.java new file mode 100644 index 000000000..4ea669ed2 --- /dev/null +++ b/examples/storm-starter/src/jvm/storm/starter/util/TupleHelpers.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.util; + +import backtype.storm.Constants; +import backtype.storm.tuple.Tuple; + +public final class TupleHelpers { + + private TupleHelpers() { + } + + public static boolean isTickTuple(Tuple tuple) { + return tuple.getSourceComponent().equals(Constants.SYSTEM_COMPONENT_ID) && tuple.getSourceStreamId().equals( + Constants.SYSTEM_TICK_STREAM_ID); + } + +} diff --git a/examples/storm-starter/test/jvm/storm/starter/bolt/IntermediateRankingsBoltTest.java b/examples/storm-starter/test/jvm/storm/starter/bolt/IntermediateRankingsBoltTest.java new file mode 100644 index 000000000..c296a899e --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/bolt/IntermediateRankingsBoltTest.java @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.Config; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import com.google.common.collect.Lists; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import storm.starter.tools.MockTupleHelpers; + +import java.util.Map; + +import static org.fest.assertions.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +public class IntermediateRankingsBoltTest { + + private static final String ANY_NON_SYSTEM_COMPONENT_ID = "irrelevant_component_id"; + private static final String ANY_NON_SYSTEM_STREAM_ID = "irrelevant_stream_id"; + private static final Object ANY_OBJECT = new Object(); + private static final int ANY_TOPN = 10; + private static final long ANY_COUNT = 42; + + private Tuple mockRankableTuple(Object obj, long count) { + Tuple tuple = MockTupleHelpers.mockTuple(ANY_NON_SYSTEM_COMPONENT_ID, ANY_NON_SYSTEM_STREAM_ID); + when(tuple.getValues()).thenReturn(Lists.newArrayList(ANY_OBJECT, ANY_COUNT)); + return tuple; + } + + @DataProvider + public Object[][] illegalTopN() { + return new Object[][]{ { -10 }, { -3 }, { -2 }, { -1 }, { 0 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalTopN") + public void negativeOrZeroTopNShouldThrowIAE(int topN) { + new IntermediateRankingsBolt(topN); + } + + @DataProvider + public Object[][] illegalEmitFrequency() { + return new Object[][]{ { -10 }, { -3 }, { -2 }, { -1 }, { 0 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalEmitFrequency") + public void negativeOrZeroEmitFrequencyShouldThrowIAE(int emitFrequencyInSeconds) { + new IntermediateRankingsBolt(ANY_TOPN, emitFrequencyInSeconds); + } + + @DataProvider + public Object[][] legalTopN() { + return new Object[][]{ { 1 }, { 2 }, { 3 }, { 20 } }; + } + + @Test(dataProvider = "legalTopN") + public void positiveTopNShouldBeOk(int topN) { + new IntermediateRankingsBolt(topN); + } + + @DataProvider + public Object[][] legalEmitFrequency() { + return new Object[][]{ { 1 }, { 2 }, { 3 }, { 20 } }; + } + + @Test(dataProvider = "legalEmitFrequency") + public void positiveEmitFrequencyShouldBeOk(int emitFrequencyInSeconds) { + new IntermediateRankingsBolt(ANY_TOPN, emitFrequencyInSeconds); + } + + @Test + public void shouldEmitSomethingIfTickTupleIsReceived() { + // given + Tuple tickTuple = MockTupleHelpers.mockTickTuple(); + BasicOutputCollector collector = mock(BasicOutputCollector.class); + IntermediateRankingsBolt bolt = new IntermediateRankingsBolt(); + + // when + bolt.execute(tickTuple, collector); + + // then + // verifyZeroInteractions(collector); + verify(collector).emit(any(Values.class)); + } + + @Test + public void shouldEmitNothingIfNormalTupleIsReceived() { + // given + Tuple normalTuple = mockRankableTuple(ANY_OBJECT, ANY_COUNT); + BasicOutputCollector collector = mock(BasicOutputCollector.class); + IntermediateRankingsBolt bolt = new IntermediateRankingsBolt(); + + // when + bolt.execute(normalTuple, collector); + + // then + verifyZeroInteractions(collector); + } + + @Test + public void shouldDeclareOutputFields() { + // given + OutputFieldsDeclarer declarer = mock(OutputFieldsDeclarer.class); + IntermediateRankingsBolt bolt = new IntermediateRankingsBolt(); + + // when + bolt.declareOutputFields(declarer); + + // then + verify(declarer, times(1)).declare(any(Fields.class)); + } + + @Test + public void shouldSetTickTupleFrequencyInComponentConfigurationToNonZeroValue() { + // given + IntermediateRankingsBolt bolt = new IntermediateRankingsBolt(); + + // when + Map componentConfig = bolt.getComponentConfiguration(); + + // then + assertThat(componentConfig).containsKey(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS); + Integer emitFrequencyInSeconds = (Integer) componentConfig.get(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS); + assertThat(emitFrequencyInSeconds).isGreaterThan(0); + } +} diff --git a/examples/storm-starter/test/jvm/storm/starter/bolt/RollingCountBoltTest.java b/examples/storm-starter/test/jvm/storm/starter/bolt/RollingCountBoltTest.java new file mode 100644 index 000000000..bc31ba06d --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/bolt/RollingCountBoltTest.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.Config; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.testng.annotations.Test; +import storm.starter.tools.MockTupleHelpers; + +import java.util.Map; + +import static org.fest.assertions.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +public class RollingCountBoltTest { + + private static final String ANY_NON_SYSTEM_COMPONENT_ID = "irrelevant_component_id"; + private static final String ANY_NON_SYSTEM_STREAM_ID = "irrelevant_stream_id"; + + private Tuple mockNormalTuple(Object obj) { + Tuple tuple = MockTupleHelpers.mockTuple(ANY_NON_SYSTEM_COMPONENT_ID, ANY_NON_SYSTEM_STREAM_ID); + when(tuple.getValue(0)).thenReturn(obj); + return tuple; + } + + @SuppressWarnings("rawtypes") + @Test + public void shouldEmitNothingIfNoObjectHasBeenCountedYetAndTickTupleIsReceived() { + // given + Tuple tickTuple = MockTupleHelpers.mockTickTuple(); + RollingCountBolt bolt = new RollingCountBolt(); + Map conf = mock(Map.class); + TopologyContext context = mock(TopologyContext.class); + OutputCollector collector = mock(OutputCollector.class); + bolt.prepare(conf, context, collector); + + // when + bolt.execute(tickTuple); + + // then + verifyZeroInteractions(collector); + } + + @SuppressWarnings("rawtypes") + @Test + public void shouldEmitSomethingIfAtLeastOneObjectWasCountedAndTickTupleIsReceived() { + // given + Tuple normalTuple = mockNormalTuple(new Object()); + Tuple tickTuple = MockTupleHelpers.mockTickTuple(); + + RollingCountBolt bolt = new RollingCountBolt(); + Map conf = mock(Map.class); + TopologyContext context = mock(TopologyContext.class); + OutputCollector collector = mock(OutputCollector.class); + bolt.prepare(conf, context, collector); + + // when + bolt.execute(normalTuple); + bolt.execute(tickTuple); + + // then + verify(collector).emit(any(Values.class)); + } + + @Test + public void shouldDeclareOutputFields() { + // given + OutputFieldsDeclarer declarer = mock(OutputFieldsDeclarer.class); + RollingCountBolt bolt = new RollingCountBolt(); + + // when + bolt.declareOutputFields(declarer); + + // then + verify(declarer, times(1)).declare(any(Fields.class)); + + } + + @Test + public void shouldSetTickTupleFrequencyInComponentConfigurationToNonZeroValue() { + // given + RollingCountBolt bolt = new RollingCountBolt(); + + // when + Map componentConfig = bolt.getComponentConfiguration(); + + // then + assertThat(componentConfig).containsKey(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS); + Integer emitFrequencyInSeconds = (Integer) componentConfig.get(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS); + assertThat(emitFrequencyInSeconds).isGreaterThan(0); + } +} diff --git a/examples/storm-starter/test/jvm/storm/starter/bolt/TotalRankingsBoltTest.java b/examples/storm-starter/test/jvm/storm/starter/bolt/TotalRankingsBoltTest.java new file mode 100644 index 000000000..49e3d679f --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/bolt/TotalRankingsBoltTest.java @@ -0,0 +1,147 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.bolt; + +import backtype.storm.Config; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import storm.starter.tools.MockTupleHelpers; +import storm.starter.tools.Rankings; + +import java.util.Map; + +import static org.fest.assertions.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +public class TotalRankingsBoltTest { + + private static final String ANY_NON_SYSTEM_COMPONENT_ID = "irrelevant_component_id"; + private static final String ANY_NON_SYSTEM_STREAM_ID = "irrelevant_stream_id"; + private static final Object ANY_OBJECT = new Object(); + private static final int ANY_TOPN = 10; + private static final long ANY_COUNT = 42; + + private Tuple mockRankingsTuple(Object obj, long count) { + Tuple tuple = MockTupleHelpers.mockTuple(ANY_NON_SYSTEM_COMPONENT_ID, ANY_NON_SYSTEM_STREAM_ID); + Rankings rankings = mock(Rankings.class); + when(tuple.getValue(0)).thenReturn(rankings); + return tuple; + } + + @DataProvider + public Object[][] illegalTopN() { + return new Object[][]{ { -10 }, { -3 }, { -2 }, { -1 }, { 0 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalTopN") + public void negativeOrZeroTopNShouldThrowIAE(int topN) { + new TotalRankingsBolt(topN); + } + + @DataProvider + public Object[][] illegalEmitFrequency() { + return new Object[][]{ { -10 }, { -3 }, { -2 }, { -1 }, { 0 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalEmitFrequency") + public void negativeOrZeroEmitFrequencyShouldThrowIAE(int emitFrequencyInSeconds) { + new TotalRankingsBolt(ANY_TOPN, emitFrequencyInSeconds); + } + + @DataProvider + public Object[][] legalTopN() { + return new Object[][]{ { 1 }, { 2 }, { 3 }, { 20 } }; + } + + @Test(dataProvider = "legalTopN") + public void positiveTopNShouldBeOk(int topN) { + new TotalRankingsBolt(topN); + } + + @DataProvider + public Object[][] legalEmitFrequency() { + return new Object[][]{ { 1 }, { 2 }, { 3 }, { 20 } }; + } + + @Test(dataProvider = "legalEmitFrequency") + public void positiveEmitFrequencyShouldBeOk(int emitFrequencyInSeconds) { + new TotalRankingsBolt(ANY_TOPN, emitFrequencyInSeconds); + } + + @Test + public void shouldEmitSomethingIfTickTupleIsReceived() { + // given + Tuple tickTuple = MockTupleHelpers.mockTickTuple(); + BasicOutputCollector collector = mock(BasicOutputCollector.class); + TotalRankingsBolt bolt = new TotalRankingsBolt(); + + // when + bolt.execute(tickTuple, collector); + + // then + // verifyZeroInteractions(collector); + verify(collector).emit(any(Values.class)); + } + + @Test + public void shouldEmitNothingIfNormalTupleIsReceived() { + // given + Tuple normalTuple = mockRankingsTuple(ANY_OBJECT, ANY_COUNT); + BasicOutputCollector collector = mock(BasicOutputCollector.class); + TotalRankingsBolt bolt = new TotalRankingsBolt(); + + // when + bolt.execute(normalTuple, collector); + + // then + verifyZeroInteractions(collector); + } + + @Test + public void shouldDeclareOutputFields() { + // given + OutputFieldsDeclarer declarer = mock(OutputFieldsDeclarer.class); + TotalRankingsBolt bolt = new TotalRankingsBolt(); + + // when + bolt.declareOutputFields(declarer); + + // then + verify(declarer, times(1)).declare(any(Fields.class)); + } + + @Test + public void shouldSetTickTupleFrequencyInComponentConfigurationToNonZeroValue() { + // given + TotalRankingsBolt bolt = new TotalRankingsBolt(); + + // when + Map componentConfig = bolt.getComponentConfiguration(); + + // then + assertThat(componentConfig).containsKey(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS); + Integer emitFrequencyInSeconds = (Integer) componentConfig.get(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS); + assertThat(emitFrequencyInSeconds).isGreaterThan(0); + } +} diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java b/examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java new file mode 100644 index 000000000..b253350ef --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import backtype.storm.Constants; +import backtype.storm.tuple.Tuple; + +import static org.mockito.Mockito.*; + +public final class MockTupleHelpers { + + private MockTupleHelpers() { + } + + public static Tuple mockTickTuple() { + return mockTuple(Constants.SYSTEM_COMPONENT_ID, Constants.SYSTEM_TICK_STREAM_ID); + } + + public static Tuple mockTuple(String componentId, String streamId) { + Tuple tuple = mock(Tuple.class); + when(tuple.getSourceComponent()).thenReturn(componentId); + when(tuple.getSourceStreamId()).thenReturn(streamId); + return tuple; + } +} diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/NthLastModifiedTimeTrackerTest.java b/examples/storm-starter/test/jvm/storm/starter/tools/NthLastModifiedTimeTrackerTest.java new file mode 100644 index 000000000..fe4d987d8 --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/tools/NthLastModifiedTimeTrackerTest.java @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import backtype.storm.utils.Time; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import static org.fest.assertions.api.Assertions.assertThat; + +public class NthLastModifiedTimeTrackerTest { + + private static final int ANY_NUM_TIMES_TO_TRACK = 3; + private static final int MILLIS_IN_SEC = 1000; + + @DataProvider + public Object[][] illegalNumTimesData() { + return new Object[][]{ { -10 }, { -3 }, { -2 }, { -1 }, { 0 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalNumTimesData") + public void negativeOrZeroNumTimesToTrackShouldThrowIAE(int numTimesToTrack) { + new NthLastModifiedTimeTracker(numTimesToTrack); + } + + @DataProvider + public Object[][] legalNumTimesData() { + return new Object[][]{ { 1 }, { 2 }, { 3 }, { 20 } }; + } + + @Test(dataProvider = "legalNumTimesData") + public void positiveNumTimesToTrackShouldBeOk(int numTimesToTrack) { + new NthLastModifiedTimeTracker(numTimesToTrack); + } + + @DataProvider + public Object[][] whenNotYetMarkedAsModifiedData() { + return new Object[][]{ { 0 }, { 1 }, { 2 }, { 3 }, { 4 }, { 5 }, { 8 }, { 10 } }; + } + + @Test(dataProvider = "whenNotYetMarkedAsModifiedData") + public void shouldReturnCorrectModifiedTimeEvenWhenNotYetMarkedAsModified(int secondsToAdvance) { + // given + Time.startSimulating(); + NthLastModifiedTimeTracker tracker = new NthLastModifiedTimeTracker(ANY_NUM_TIMES_TO_TRACK); + + // when + advanceSimulatedTimeBy(secondsToAdvance); + int seconds = tracker.secondsSinceOldestModification(); + + // then + assertThat(seconds).isEqualTo(secondsToAdvance); + + // cleanup + Time.stopSimulating(); + } + + @DataProvider + public Object[][] simulatedTrackerIterations() { + return new Object[][]{ { 1, new int[]{ 0, 1 }, new int[]{ 0, 0 } }, { 1, new int[]{ 0, 2 }, new int[]{ 0, 0 } }, + { 2, new int[]{ 2, 2 }, new int[]{ 2, 2 } }, { 2, new int[]{ 0, 4 }, new int[]{ 0, 4 } }, + { 1, new int[]{ 1, 1, 1, 1, 1, 1, 1 }, new int[]{ 0, 0, 0, 0, 0, 0, 0 } }, + { 1, new int[]{ 1, 2, 3, 4, 5, 6, 7 }, new int[]{ 0, 0, 0, 0, 0, 0, 0 } }, + { 2, new int[]{ 1, 1, 1, 1, 1, 1, 1 }, new int[]{ 1, 1, 1, 1, 1, 1, 1 } }, + { 2, new int[]{ 2, 2, 2, 2, 2, 2, 2 }, new int[]{ 2, 2, 2, 2, 2, 2, 2 } }, + { 2, new int[]{ 1, 2, 3, 4, 5, 6, 7 }, new int[]{ 1, 2, 3, 4, 5, 6, 7 } }, + { 3, new int[]{ 1, 1, 1, 1, 1, 1, 1 }, new int[]{ 1, 2, 2, 2, 2, 2, 2 } }, + { 3, new int[]{ 1, 2, 3, 4, 5, 6, 7 }, new int[]{ 1, 3, 5, 7, 9, 11, 13 } }, + { 3, new int[]{ 2, 2, 2, 2, 2, 2, 2 }, new int[]{ 2, 4, 4, 4, 4, 4, 4 } }, + { 4, new int[]{ 1, 1, 1, 1, 1, 1, 1 }, new int[]{ 1, 2, 3, 3, 3, 3, 3 } }, + { 4, new int[]{ 1, 2, 3, 4, 5, 6, 7 }, new int[]{ 1, 3, 6, 9, 12, 15, 18 } }, + { 4, new int[]{ 2, 2, 2, 2, 2, 2, 2 }, new int[]{ 2, 4, 6, 6, 6, 6, 6 } }, + { 5, new int[]{ 1, 1, 1, 1, 1, 1, 1 }, new int[]{ 1, 2, 3, 4, 4, 4, 4 } }, + { 5, new int[]{ 1, 2, 3, 4, 5, 6, 7 }, new int[]{ 1, 3, 6, 10, 14, 18, 22 } }, + { 5, new int[]{ 2, 2, 2, 2, 2, 2, 2 }, new int[]{ 2, 4, 6, 8, 8, 8, 8 } }, + { 6, new int[]{ 1, 1, 1, 1, 1, 1, 1 }, new int[]{ 1, 2, 3, 4, 5, 5, 5 } }, + { 6, new int[]{ 1, 2, 3, 4, 5, 6, 7 }, new int[]{ 1, 3, 6, 10, 15, 20, 25 } }, + { 6, new int[]{ 2, 2, 2, 2, 2, 2, 2 }, new int[]{ 2, 4, 6, 8, 10, 10, 10 } }, + { 3, new int[]{ 1, 2, 3 }, new int[]{ 1, 3, 5 } } }; + } + + @Test(dataProvider = "simulatedTrackerIterations") + public void shouldReturnCorrectModifiedTimeWhenMarkedAsModified(int numTimesToTrack, + int[] secondsToAdvancePerIteration, int[] expLastModifiedTimes) { + // given + Time.startSimulating(); + NthLastModifiedTimeTracker tracker = new NthLastModifiedTimeTracker(numTimesToTrack); + + int[] modifiedTimes = new int[expLastModifiedTimes.length]; + + // when + int i = 0; + for (int secondsToAdvance : secondsToAdvancePerIteration) { + advanceSimulatedTimeBy(secondsToAdvance); + tracker.markAsModified(); + modifiedTimes[i] = tracker.secondsSinceOldestModification(); + i++; + } + + // then + assertThat(modifiedTimes).isEqualTo(expLastModifiedTimes); + + // cleanup + Time.stopSimulating(); + } + + private void advanceSimulatedTimeBy(int seconds) { + Time.advanceTime(seconds * MILLIS_IN_SEC); + } +} diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/RankableObjectWithFieldsTest.java b/examples/storm-starter/test/jvm/storm/starter/tools/RankableObjectWithFieldsTest.java new file mode 100644 index 000000000..e83f92209 --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/tools/RankableObjectWithFieldsTest.java @@ -0,0 +1,252 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import backtype.storm.tuple.Tuple; +import com.google.common.collect.Lists; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.fest.assertions.api.Assertions.assertThat; +import static org.mockito.Mockito.*; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +public class RankableObjectWithFieldsTest { + + private static final Object ANY_OBJECT = new Object(); + private static final long ANY_COUNT = 271; + private static final String ANY_FIELD = "someAdditionalField"; + private static final int GREATER_THAN = 1; + private static final int EQUAL_TO = 0; + private static final int SMALLER_THAN = -1; + + @Test(expectedExceptions = IllegalArgumentException.class) + public void constructorWithNullObjectAndNoFieldsShouldThrowIAE() { + new RankableObjectWithFields(null, ANY_COUNT); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void constructorWithNullObjectAndFieldsShouldThrowIAE() { + Object someAdditionalField = new Object(); + new RankableObjectWithFields(null, ANY_COUNT, someAdditionalField); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void constructorWithNegativeCountAndNoFieldsShouldThrowIAE() { + new RankableObjectWithFields(ANY_OBJECT, -1); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void constructorWithNegativeCountAndFieldsShouldThrowIAE() { + Object someAdditionalField = new Object(); + new RankableObjectWithFields(ANY_OBJECT, -1, someAdditionalField); + } + + @Test + public void shouldBeEqualToItself() { + RankableObjectWithFields r = new RankableObjectWithFields(ANY_OBJECT, ANY_COUNT); + assertThat(r).isEqualTo(r); + } + + @DataProvider + public Object[][] otherClassesData() { + return new Object[][]{ { new String("foo") }, { new Object() }, { Integer.valueOf(4) }, { Lists.newArrayList(7, 8, + 9) } }; + } + + @Test(dataProvider = "otherClassesData") + public void shouldNotBeEqualToInstancesOfOtherClasses(Object notARankable) { + RankableObjectWithFields r = new RankableObjectWithFields(ANY_OBJECT, ANY_COUNT); + assertFalse(r.equals(notARankable), r + " is equal to " + notARankable + " but it should not be"); + } + + @DataProvider + public Object[][] falseDuplicatesData() { + return new Object[][]{ { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("foo", 1) }, + { new RankableObjectWithFields("foo", 1), new RankableObjectWithFields("Foo", 1) }, + { new RankableObjectWithFields("foo", 1), new RankableObjectWithFields("FOO", 1) }, + { new RankableObjectWithFields("foo", 1), new RankableObjectWithFields("bar", 1) }, + { new RankableObjectWithFields("", 0), new RankableObjectWithFields("", 1) }, { new RankableObjectWithFields("", + 1), new RankableObjectWithFields("bar", 1) } }; + } + + @Test(dataProvider = "falseDuplicatesData") + public void shouldNotBeEqualToFalseDuplicates(RankableObjectWithFields r, RankableObjectWithFields falseDuplicate) { + assertFalse(r.equals(falseDuplicate), r + " is equal to " + falseDuplicate + " but it should not be"); + } + + @Test(dataProvider = "falseDuplicatesData") + public void shouldHaveDifferentHashCodeThanFalseDuplicates(RankableObjectWithFields r, + RankableObjectWithFields falseDuplicate) { + assertThat(r.hashCode()).isNotEqualTo(falseDuplicate.hashCode()); + } + + @DataProvider + public Object[][] trueDuplicatesData() { + return new Object[][]{ { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("foo", 0) }, + { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("foo", 0, "someOtherField") }, + { new RankableObjectWithFields("foo", 0, "someField"), new RankableObjectWithFields("foo", 0, + "someOtherField") } }; + } + + @Test(dataProvider = "trueDuplicatesData") + public void shouldBeEqualToTrueDuplicates(RankableObjectWithFields r, RankableObjectWithFields trueDuplicate) { + assertTrue(r.equals(trueDuplicate), r + " is not equal to " + trueDuplicate + " but it should be"); + } + + @Test(dataProvider = "trueDuplicatesData") + public void shouldHaveSameHashCodeAsTrueDuplicates(RankableObjectWithFields r, + RankableObjectWithFields trueDuplicate) { + assertThat(r.hashCode()).isEqualTo(trueDuplicate.hashCode()); + } + + @DataProvider + public Object[][] compareToData() { + return new Object[][]{ { new RankableObjectWithFields("foo", 1000), new RankableObjectWithFields("foo", 0), + GREATER_THAN }, { new RankableObjectWithFields("foo", 1), new RankableObjectWithFields("foo", 0), + GREATER_THAN }, { new RankableObjectWithFields("foo", 1000), new RankableObjectWithFields("bar", 0), + GREATER_THAN }, { new RankableObjectWithFields("foo", 1), new RankableObjectWithFields("bar", 0), + GREATER_THAN }, { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("foo", 0), EQUAL_TO }, + { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("bar", 0), EQUAL_TO }, + { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("foo", 1000), SMALLER_THAN }, + { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("foo", 1), SMALLER_THAN }, + { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("bar", 1), SMALLER_THAN }, + { new RankableObjectWithFields("foo", 0), new RankableObjectWithFields("bar", 1000), SMALLER_THAN }, }; + } + + @Test(dataProvider = "compareToData") + public void verifyCompareTo(RankableObjectWithFields first, RankableObjectWithFields second, int expCompareToValue) { + assertThat(first.compareTo(second)).isEqualTo(expCompareToValue); + } + + @DataProvider + public Object[][] toStringData() { + return new Object[][]{ { new String("foo"), 0L }, { new String("BAR"), 8L } }; + } + + @Test(dataProvider = "toStringData") + public void toStringShouldContainStringRepresentationsOfObjectAndCount(Object obj, long count) { + // given + RankableObjectWithFields r = new RankableObjectWithFields(obj, count); + + // when + String strRepresentation = r.toString(); + + // then + assertThat(strRepresentation).contains(obj.toString()).contains("" + count); + } + + @Test + public void shouldReturnTheObject() { + // given + RankableObjectWithFields r = new RankableObjectWithFields(ANY_OBJECT, ANY_COUNT, ANY_FIELD); + + // when + Object obj = r.getObject(); + + // then + assertThat(obj).isEqualTo(ANY_OBJECT); + } + + @Test + public void shouldReturnTheCount() { + // given + RankableObjectWithFields r = new RankableObjectWithFields(ANY_OBJECT, ANY_COUNT, ANY_FIELD); + + // when + long count = r.getCount(); + + // then + assertThat(count).isEqualTo(ANY_COUNT); + } + + @DataProvider + public Object[][] fieldsData() { + return new Object[][]{ { ANY_OBJECT, ANY_COUNT, new Object[]{ ANY_FIELD } }, + { "quux", 42L, new Object[]{ "one", "two", "three" } } }; + } + + @Test(dataProvider = "fieldsData") + public void shouldReturnTheFields(Object obj, long count, Object[] fields) { + // given + RankableObjectWithFields r = new RankableObjectWithFields(obj, count, fields); + + // when + List actualFields = r.getFields(); + + // then + assertThat(actualFields).isEqualTo(Lists.newArrayList(fields)); + } + + @Test(expectedExceptions = UnsupportedOperationException.class) + public void fieldsShouldBeImmutable() { + // given + RankableObjectWithFields r = new RankableObjectWithFields(ANY_OBJECT, ANY_COUNT, ANY_FIELD); + + // when + List fields = r.getFields(); + // try to modify the list, which should fail + fields.remove(0); + + // then (exception) + } + + @Test + public void shouldCreateRankableObjectFromTuple() { + // given + Tuple tuple = mock(Tuple.class); + List tupleValues = Lists.newArrayList(ANY_OBJECT, ANY_COUNT, ANY_FIELD); + when(tuple.getValues()).thenReturn(tupleValues); + + // when + RankableObjectWithFields r = RankableObjectWithFields.from(tuple); + + // then + assertThat(r.getObject()).isEqualTo(ANY_OBJECT); + assertThat(r.getCount()).isEqualTo(ANY_COUNT); + List fields = new ArrayList(); + fields.add(ANY_FIELD); + assertThat(r.getFields()).isEqualTo(fields); + + } + + @DataProvider + public Object[][] copyData() { + return new Object[][]{ { new RankableObjectWithFields("foo", 0) }, { new RankableObjectWithFields("foo", 3, + "someOtherField") }, { new RankableObjectWithFields("foo", 0, "someField") } }; + } + + // TODO: What would be a good test to ensure that RankableObjectWithFields is at least somewhat defensively copied? + // The contract of Rankable#copy() returns a Rankable value, not a RankableObjectWithFields. + @Test(dataProvider = "copyData") + public void copyShouldReturnCopy(RankableObjectWithFields original) { + // given + + // when + Rankable copy = original.copy(); + + // then + assertThat(copy.getObject()).isEqualTo(original.getObject()); + assertThat(copy.getCount()).isEqualTo(original.getCount()); + } + +} diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/RankingsTest.java b/examples/storm-starter/test/jvm/storm/starter/tools/RankingsTest.java new file mode 100644 index 000000000..cab02cbe1 --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/tools/RankingsTest.java @@ -0,0 +1,368 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.jmock.lib.concurrent.Blitzer; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.util.List; + +import static org.fest.assertions.api.Assertions.assertThat; + +public class RankingsTest { + + private static final int ANY_TOPN = 42; + private static final Rankable ANY_RANKABLE = new RankableObjectWithFields("someObject", ANY_TOPN); + private static final Rankable ZERO = new RankableObjectWithFields("ZERO_COUNT", 0); + private static final Rankable A = new RankableObjectWithFields("A", 1); + private static final Rankable B = new RankableObjectWithFields("B", 2); + private static final Rankable C = new RankableObjectWithFields("C", 3); + private static final Rankable D = new RankableObjectWithFields("D", 4); + private static final Rankable E = new RankableObjectWithFields("E", 5); + private static final Rankable F = new RankableObjectWithFields("F", 6); + private static final Rankable G = new RankableObjectWithFields("G", 7); + private static final Rankable H = new RankableObjectWithFields("H", 8); + + @DataProvider + public Object[][] illegalTopNData() { + return new Object[][]{ { 0 }, { -1 }, { -2 }, { -10 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalTopNData") + public void constructorWithNegativeOrZeroTopNShouldThrowIAE(int topN) { + new Rankings(topN); + } + + @DataProvider + public Object[][] copyRankingsData() { + return new Object[][]{ { 5, Lists.newArrayList(A, B, C) }, { 2, Lists.newArrayList(A, B, C, D) }, + { 1, Lists.newArrayList() }, { 1, Lists.newArrayList(A) }, { 1, Lists.newArrayList(A, B) } }; + } + + @Test(dataProvider = "copyRankingsData") + public void copyConstructorShouldReturnCopy(int topN, List rankables) { + // given + Rankings rankings = new Rankings(topN); + for (Rankable r : rankables) { + rankings.updateWith(r); + } + + // when + Rankings copy = new Rankings(rankings); + + // then + assertThat(copy.maxSize()).isEqualTo(rankings.maxSize()); + assertThat(copy.getRankings()).isEqualTo(rankings.getRankings()); + } + + @DataProvider + public Object[][] defensiveCopyRankingsData() { + return new Object[][]{ { 5, Lists.newArrayList(A, B, C), Lists.newArrayList(D) }, { 2, Lists.newArrayList(A, B, C, + D), Lists.newArrayList(E, F) }, { 1, Lists.newArrayList(), Lists.newArrayList(A) }, { 1, Lists.newArrayList(A), + Lists.newArrayList(B) }, { 1, Lists.newArrayList(ZERO), Lists.newArrayList(B) }, { 1, Lists.newArrayList(ZERO), + Lists.newArrayList() } }; + } + + @Test(dataProvider = "defensiveCopyRankingsData") + public void copyConstructorShouldReturnDefensiveCopy(int topN, List rankables, List changes) { + // given + Rankings original = new Rankings(topN); + for (Rankable r : rankables) { + original.updateWith(r); + } + int expSize = original.size(); + List expRankings = original.getRankings(); + + // when + Rankings copy = new Rankings(original); + for (Rankable r : changes) { + copy.updateWith(r); + } + + // then + assertThat(original.size()).isEqualTo(expSize); + assertThat(original.getRankings()).isEqualTo(expRankings); + } + + @DataProvider + public Object[][] legalTopNData() { + return new Object[][]{ { 1 }, { 2 }, { 1000 }, { 1000000 } }; + } + + @Test(dataProvider = "legalTopNData") + public void constructorWithPositiveTopNShouldBeOk(int topN) { + // given/when + Rankings rankings = new Rankings(topN); + + // then + assertThat(rankings.maxSize()).isEqualTo(topN); + } + + @Test + public void shouldHaveDefaultConstructor() { + new Rankings(); + } + + @Test + public void defaultConstructorShouldSetPositiveTopN() { + // given/when + Rankings rankings = new Rankings(); + + // then + assertThat(rankings.maxSize()).isGreaterThan(0); + } + + @DataProvider + public Object[][] rankingsGrowData() { + return new Object[][]{ { 2, Lists.newArrayList(new RankableObjectWithFields("A", 1), new RankableObjectWithFields( + "B", 2), new RankableObjectWithFields("C", 3)) }, { 2, Lists.newArrayList(new RankableObjectWithFields("A", 1), + new RankableObjectWithFields("B", 2), new RankableObjectWithFields("C", 3), new RankableObjectWithFields("D", + 4)) } }; + } + + @Test(dataProvider = "rankingsGrowData") + public void sizeOfRankingsShouldNotGrowBeyondTopN(int topN, List rankables) { + // sanity check of the provided test data + assertThat(rankables.size()).overridingErrorMessage( + "The supplied test data is not correct: the number of rankables <%d> should be greater than <%d>", + rankables.size(), topN).isGreaterThan(topN); + + // given + Rankings rankings = new Rankings(topN); + + // when + for (Rankable r : rankables) { + rankings.updateWith(r); + } + + // then + assertThat(rankings.size()).isLessThanOrEqualTo(rankings.maxSize()); + } + + @DataProvider + public Object[][] simulatedRankingsData() { + return new Object[][]{ { Lists.newArrayList(A), Lists.newArrayList(A) }, { Lists.newArrayList(B, D, A, C), + Lists.newArrayList(D, C, B, A) }, { Lists.newArrayList(B, F, A, C, D, E), Lists.newArrayList(F, E, D, C, B, + A) }, { Lists.newArrayList(G, B, F, A, C, D, E, H), Lists.newArrayList(H, G, F, E, D, C, B, A) } }; + } + + @Test(dataProvider = "simulatedRankingsData") + public void shouldCorrectlyRankWhenUpdatedWithRankables(List unsorted, List expSorted) { + // given + Rankings rankings = new Rankings(unsorted.size()); + + // when + for (Rankable r : unsorted) { + rankings.updateWith(r); + } + + // then + assertThat(rankings.getRankings()).isEqualTo(expSorted); + } + + @Test(dataProvider = "simulatedRankingsData") + public void shouldCorrectlyRankWhenEmptyAndUpdatedWithOtherRankings(List unsorted, + List expSorted) { + // given + Rankings rankings = new Rankings(unsorted.size()); + Rankings otherRankings = new Rankings(rankings.maxSize()); + for (Rankable r : unsorted) { + otherRankings.updateWith(r); + } + + // when + rankings.updateWith(otherRankings); + + // then + assertThat(rankings.getRankings()).isEqualTo(expSorted); + } + + @Test(dataProvider = "simulatedRankingsData") + public void shouldCorrectlyRankWhenUpdatedWithEmptyOtherRankings(List unsorted, List expSorted) { + // given + Rankings rankings = new Rankings(unsorted.size()); + for (Rankable r : unsorted) { + rankings.updateWith(r); + } + Rankings emptyRankings = new Rankings(ANY_TOPN); + + // when + rankings.updateWith(emptyRankings); + + // then + assertThat(rankings.getRankings()).isEqualTo(expSorted); + } + + @DataProvider + public Object[][] simulatedRankingsAndOtherRankingsData() { + return new Object[][]{ { Lists.newArrayList(A), Lists.newArrayList(A), Lists.newArrayList(A) }, + { Lists.newArrayList(A, C), Lists.newArrayList(B, D), Lists.newArrayList(D, C, B, A) }, { Lists.newArrayList(B, + F, A), Lists.newArrayList(C, D, E), Lists.newArrayList(F, E, D, C, B, A) }, { Lists.newArrayList(G, B, F, A, C), + Lists.newArrayList(D, E, H), Lists.newArrayList(H, G, F, E, D, C, B, A) } }; + } + + @Test(dataProvider = "simulatedRankingsAndOtherRankingsData") + public void shouldCorrectlyRankWhenNotEmptyAndUpdatedWithOtherRankings(List unsorted, + List unsortedForOtherRankings, List expSorted) { + // given + Rankings rankings = new Rankings(expSorted.size()); + for (Rankable r : unsorted) { + rankings.updateWith(r); + } + Rankings otherRankings = new Rankings(unsortedForOtherRankings.size()); + for (Rankable r : unsortedForOtherRankings) { + otherRankings.updateWith(r); + } + + // when + rankings.updateWith(otherRankings); + + // then + assertThat(rankings.getRankings()).isEqualTo(expSorted); + } + + @DataProvider + public Object[][] duplicatesData() { + Rankable A1 = new RankableObjectWithFields("A", 1); + Rankable A2 = new RankableObjectWithFields("A", 2); + Rankable A3 = new RankableObjectWithFields("A", 3); + return new Object[][]{ { Lists.newArrayList(ANY_RANKABLE, ANY_RANKABLE, ANY_RANKABLE) }, { Lists.newArrayList(A1, + A2, A3) }, }; + } + + @Test(dataProvider = "duplicatesData") + public void shouldNotRankDuplicateObjectsMoreThanOnce(List duplicates) { + // given + Rankings rankings = new Rankings(duplicates.size()); + + // when + for (Rankable r : duplicates) { + rankings.updateWith(r); + } + + // then + assertThat(rankings.size()).isEqualTo(1); + } + + @DataProvider + public Object[][] removeZeroRankingsData() { + return new Object[][]{ { Lists.newArrayList(A, ZERO), Lists.newArrayList(A) }, { Lists.newArrayList(A), + Lists.newArrayList(A) }, { Lists.newArrayList(ZERO, A), Lists.newArrayList(A) }, { Lists.newArrayList(ZERO), + Lists.newArrayList() }, { Lists.newArrayList(ZERO, new RankableObjectWithFields("ZERO2", 0)), + Lists.newArrayList() }, { Lists.newArrayList(B, ZERO, new RankableObjectWithFields("ZERO2", 0), D, + new RankableObjectWithFields("ZERO3", 0), new RankableObjectWithFields("ZERO4", 0), C), Lists.newArrayList(D, C, + B) }, { Lists.newArrayList(A, ZERO, B), Lists.newArrayList(B, A) } }; + } + + @Test(dataProvider = "removeZeroRankingsData") + public void shouldRemoveZeroCounts(List unsorted, List expSorted) { + // given + Rankings rankings = new Rankings(unsorted.size()); + for (Rankable r : unsorted) { + rankings.updateWith(r); + } + + // when + rankings.pruneZeroCounts(); + + // then + assertThat(rankings.getRankings()).isEqualTo(expSorted); + } + + @Test + public void updatingWithNewRankablesShouldBeThreadSafe() throws InterruptedException { + // given + final List entries = ImmutableList.of(A, B, C, D); + final Rankings rankings = new Rankings(entries.size()); + + // We are capturing exceptions thrown in Blitzer's child threads into this data structure so that we can properly + // pass/fail this test. The reason is that Blitzer doesn't report exceptions, which is a known bug in Blitzer + // (JMOCK-263). See https://github.com/jmock-developers/jmock-library/issues/22 for more information. + final List exceptions = Lists.newArrayList(); + Blitzer blitzer = new Blitzer(1000); + + // when + blitzer.blitz(new Runnable() { + public void run() { + for (Rankable r : entries) { + try { + rankings.updateWith(r); + } + catch (RuntimeException e) { + synchronized(exceptions) { + exceptions.add(e); + } + } + } + } + }); + blitzer.shutdown(); + + // then + // + if (!exceptions.isEmpty()) { + for (Exception e : exceptions) { + System.err.println(Throwables.getStackTraceAsString(e)); + } + } + assertThat(exceptions).isEmpty(); + } + + @Test(dataProvider = "copyRankingsData") + public void copyShouldReturnCopy(int topN, List rankables) { + // given + Rankings rankings = new Rankings(topN); + for (Rankable r : rankables) { + rankings.updateWith(r); + } + + // when + Rankings copy = rankings.copy(); + + // then + assertThat(copy.maxSize()).isEqualTo(rankings.maxSize()); + assertThat(copy.getRankings()).isEqualTo(rankings.getRankings()); + } + + @Test(dataProvider = "defensiveCopyRankingsData") + public void copyShouldReturnDefensiveCopy(int topN, List rankables, List changes) { + // given + Rankings original = new Rankings(topN); + for (Rankable r : rankables) { + original.updateWith(r); + } + int expSize = original.size(); + List expRankings = original.getRankings(); + + // when + Rankings copy = original.copy(); + for (Rankable r : changes) { + copy.updateWith(r); + } + copy.pruneZeroCounts(); + + // then + assertThat(original.size()).isEqualTo(expSize); + assertThat(original.getRankings()).isEqualTo(expRankings); + } + +} \ No newline at end of file diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/SlidingWindowCounterTest.java b/examples/storm-starter/test/jvm/storm/starter/tools/SlidingWindowCounterTest.java new file mode 100644 index 000000000..920bf017e --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/tools/SlidingWindowCounterTest.java @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.util.Map; + +import static org.fest.assertions.api.Assertions.assertThat; + +public class SlidingWindowCounterTest { + + private static final int ANY_WINDOW_LENGTH_IN_SLOTS = 2; + private static final Object ANY_OBJECT = "ANY_OBJECT"; + + @DataProvider + public Object[][] illegalWindowLengths() { + return new Object[][]{ { -10 }, { -3 }, { -2 }, { -1 }, { 0 }, { 1 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalWindowLengths") + public void lessThanTwoSlotsShouldThrowIAE(int windowLengthInSlots) { + new SlidingWindowCounter(windowLengthInSlots); + } + + @DataProvider + public Object[][] legalWindowLengths() { + return new Object[][]{ { 2 }, { 3 }, { 20 } }; + } + + @Test(dataProvider = "legalWindowLengths") + public void twoOrMoreSlotsShouldBeValid(int windowLengthInSlots) { + new SlidingWindowCounter(windowLengthInSlots); + } + + @Test + public void newInstanceShouldHaveEmptyCounts() { + // given + SlidingWindowCounter counter = new SlidingWindowCounter(ANY_WINDOW_LENGTH_IN_SLOTS); + + // when + Map counts = counter.getCountsThenAdvanceWindow(); + + // then + assertThat(counts).isEmpty(); + } + + @DataProvider + public Object[][] simulatedCounterIterations() { + return new Object[][]{ { 2, new int[]{ 3, 2, 0, 0, 1, 0, 0, 0 }, new long[]{ 3, 5, 2, 0, 1, 1, 0, 0 } }, + { 3, new int[]{ 3, 2, 0, 0, 1, 0, 0, 0 }, new long[]{ 3, 5, 5, 2, 1, 1, 1, 0 } }, + { 4, new int[]{ 3, 2, 0, 0, 1, 0, 0, 0 }, new long[]{ 3, 5, 5, 5, 3, 1, 1, 1 } }, + { 5, new int[]{ 3, 2, 0, 0, 1, 0, 0, 0 }, new long[]{ 3, 5, 5, 5, 6, 3, 1, 1 } }, + { 5, new int[]{ 3, 11, 5, 13, 7, 17, 0, 3, 50, 600, 7000 }, + new long[]{ 3, 14, 19, 32, 39, 53, 42, 40, 77, 670, 7653 } }, }; + } + + @Test(dataProvider = "simulatedCounterIterations") + public void testCounterWithSimulatedRuns(int windowLengthInSlots, int[] incrementsPerIteration, + long[] expCountsPerIteration) { + // given + SlidingWindowCounter counter = new SlidingWindowCounter(windowLengthInSlots); + int numIterations = incrementsPerIteration.length; + + for (int i = 0; i < numIterations; i++) { + int numIncrements = incrementsPerIteration[i]; + long expCounts = expCountsPerIteration[i]; + // Objects are absent if they were zero both this iteration + // and the last -- if only this one, we need to report zero. + boolean expAbsent = ((expCounts == 0) && ((i == 0) || (expCountsPerIteration[i - 1] == 0))); + + // given (for this iteration) + for (int j = 0; j < numIncrements; j++) { + counter.incrementCount(ANY_OBJECT); + } + + // when (for this iteration) + Map counts = counter.getCountsThenAdvanceWindow(); + + // then (for this iteration) + if (expAbsent) { + assertThat(counts).doesNotContainKey(ANY_OBJECT); + } + else { + assertThat(counts.get(ANY_OBJECT)).isEqualTo(expCounts); + } + } + } + +} diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/SlotBasedCounterTest.java b/examples/storm-starter/test/jvm/storm/starter/tools/SlotBasedCounterTest.java new file mode 100644 index 000000000..3ad042beb --- /dev/null +++ b/examples/storm-starter/test/jvm/storm/starter/tools/SlotBasedCounterTest.java @@ -0,0 +1,181 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.starter.tools; + +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.util.Map; + +import static org.fest.assertions.api.Assertions.assertThat; + +public class SlotBasedCounterTest { + + private static final int ANY_NUM_SLOTS = 1; + private static final int ANY_SLOT = 0; + private static final Object ANY_OBJECT = "ANY_OBJECT"; + + @DataProvider + public Object[][] illegalNumSlotsData() { + return new Object[][]{ { -10 }, { -3 }, { -2 }, { -1 }, { 0 } }; + } + + @Test(expectedExceptions = IllegalArgumentException.class, dataProvider = "illegalNumSlotsData") + public void negativeOrZeroNumSlotsShouldThrowIAE(int numSlots) { + new SlotBasedCounter(numSlots); + } + + @DataProvider + public Object[][] legalNumSlotsData() { + return new Object[][]{ { 1 }, { 2 }, { 3 }, { 20 } }; + } + + @Test(dataProvider = "legalNumSlotsData") + public void positiveNumSlotsShouldBeOk(int numSlots) { + new SlotBasedCounter(numSlots); + } + + @Test + public void newInstanceShouldHaveEmptyCounts() { + // given + SlotBasedCounter counter = new SlotBasedCounter(ANY_NUM_SLOTS); + + // when + Map counts = counter.getCounts(); + + // then + assertThat(counts).isEmpty(); + } + + @Test + public void shouldReturnNonEmptyCountsWhenAtLeastOneObjectWasCounted() { + // given + SlotBasedCounter counter = new SlotBasedCounter(ANY_NUM_SLOTS); + counter.incrementCount(ANY_OBJECT, ANY_SLOT); + + // when + Map counts = counter.getCounts(); + + // then + assertThat(counts).isNotEmpty(); + + // additional tests that go beyond what this test is primarily about + assertThat(counts.size()).isEqualTo(1); + assertThat(counts.get(ANY_OBJECT)).isEqualTo(1); + } + + @DataProvider + public Object[][] incrementCountData() { + return new Object[][]{ { new String[]{ "foo", "bar" }, new int[]{ 3, 2 } } }; + } + + @Test(dataProvider = "incrementCountData") + public void shouldIncrementCount(Object[] objects, int[] expCounts) { + // given + SlotBasedCounter counter = new SlotBasedCounter(ANY_NUM_SLOTS); + + // when + for (int i = 0; i < objects.length; i++) { + Object obj = objects[i]; + int numIncrements = expCounts[i]; + for (int j = 0; j < numIncrements; j++) { + counter.incrementCount(obj, ANY_SLOT); + } + } + + // then + for (int i = 0; i < objects.length; i++) { + assertThat(counter.getCount(objects[i], ANY_SLOT)).isEqualTo(expCounts[i]); + } + assertThat(counter.getCount("nonexistentObject", ANY_SLOT)).isEqualTo(0); + } + + @Test + public void shouldReturnZeroForNonexistentObject() { + // given + SlotBasedCounter counter = new SlotBasedCounter(ANY_NUM_SLOTS); + + // when + counter.incrementCount("somethingElse", ANY_SLOT); + + // then + assertThat(counter.getCount("nonexistentObject", ANY_SLOT)).isEqualTo(0); + } + + @Test + public void shouldIncrementCountOnlyOneSlotAtATime() { + // given + int numSlots = 3; + Object obj = Long.valueOf(10); + SlotBasedCounter counter = new SlotBasedCounter(numSlots); + + // when (empty) + // then + assertThat(counter.getCount(obj, 0)).isEqualTo(0); + assertThat(counter.getCount(obj, 1)).isEqualTo(0); + assertThat(counter.getCount(obj, 2)).isEqualTo(0); + + // when + counter.incrementCount(obj, 1); + + // then + assertThat(counter.getCount(obj, 0)).isEqualTo(0); + assertThat(counter.getCount(obj, 1)).isEqualTo(1); + assertThat(counter.getCount(obj, 2)).isEqualTo(0); + } + + @Test + public void wipeSlotShouldSetAllCountsInSlotToZero() { + // given + SlotBasedCounter counter = new SlotBasedCounter(ANY_NUM_SLOTS); + Object countWasOne = "countWasOne"; + Object countWasThree = "countWasThree"; + counter.incrementCount(countWasOne, ANY_SLOT); + counter.incrementCount(countWasThree, ANY_SLOT); + counter.incrementCount(countWasThree, ANY_SLOT); + counter.incrementCount(countWasThree, ANY_SLOT); + + // when + counter.wipeSlot(ANY_SLOT); + + // then + assertThat(counter.getCount(countWasOne, ANY_SLOT)).isEqualTo(0); + assertThat(counter.getCount(countWasThree, ANY_SLOT)).isEqualTo(0); + } + + @Test + public void wipeZerosShouldRemoveAnyObjectsWithZeroTotalCount() { + // given + SlotBasedCounter counter = new SlotBasedCounter(2); + int wipeSlot = 0; + int otherSlot = 1; + Object willBeRemoved = "willBeRemoved"; + Object willContinueToBeTracked = "willContinueToBeTracked"; + counter.incrementCount(willBeRemoved, wipeSlot); + counter.incrementCount(willContinueToBeTracked, wipeSlot); + counter.incrementCount(willContinueToBeTracked, otherSlot); + + // when + counter.wipeSlot(wipeSlot); + counter.wipeZeros(); + + // then + assertThat(counter.getCounts()).doesNotContainKey(willBeRemoved); + assertThat(counter.getCounts()).containsKey(willContinueToBeTracked); + } +} diff --git a/external/storm-kafka/CHANGELOG.md b/external/storm-kafka/CHANGELOG.md new file mode 100644 index 000000000..33a49eecc --- /dev/null +++ b/external/storm-kafka/CHANGELOG.md @@ -0,0 +1,13 @@ +## 0.9.2-incubating (0.5.0) +* incorporated as an Apache Storm external module +* fixed partition assignment for KafkaSpout +* upgraded to storm 0.9.1 + +## 0.4.0 +* added support for reading kafka message keys +* configurable metrics emit interval + +## 0.3.0 +* updated partition path in zookeeper +* added error handling for fetch request + diff --git a/external/storm-kafka/README.md b/external/storm-kafka/README.md new file mode 100644 index 000000000..726cff648 --- /dev/null +++ b/external/storm-kafka/README.md @@ -0,0 +1,25 @@ +Storm Kafka +==================== + +Provides core storm and Trident spout implementations for consuming data from Apache Kafka 0.8.x. + + + +##Usage Example + +```java + + TridentTopology topology = new TridentTopology(); + + BrokerHosts zk = new ZkHosts("localhost"); + + TridentKafkaConfig spoutConf = new TridentKafkaConfig(zk, "test-topic"); + spoutConf.scheme = new SchemeAsMultiScheme(new StringScheme()); + OpaqueTridentKafkaSpout spout = new OpaqueTridentKafkaSpout(spoutConf); + + +``` + +## Committer Sponsors + + * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org)) \ No newline at end of file diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml new file mode 100644 index 000000000..938567814 --- /dev/null +++ b/external/storm-kafka/pom.xml @@ -0,0 +1,138 @@ + + + + 4.0.0 + + + storm + org.apache.storm + 0.9.2-incubating-SNAPSHOT + ../../pom.xml + + + jar + storm-kafka + storm-kafka + Storm Spouts for Apache Kafka + + 2.9.2 + kafka_2.9.2 + + + + + + + src/jvm + src/test + + + + org.mockito + mockito-all + 1.9.0 + test + + + org.scala-lang + scala-library + ${scalaVersion} + + + junit + junit + 4.11 + test + + + com.netflix.curator + curator-framework + 1.3.3 + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + com.netflix.curator + curator-recipes + 1.3.3 + + + log4j + log4j + + + test + + + com.netflix.curator + curator-test + 1.3.3 + + + log4j + log4j + + + org.testng + testng + + + test + + + org.apache.kafka + ${kafkaArtifact} + 0.8.1 + + + org.apache.zookeeper + zookeeper + + + log4j + log4j + + + + + org.apache.storm + storm-core + ${project.version} + provided + + + + + Scala-2.10 + + 2.10.3 + kafka_2.10 + scala_2.10 + + + + diff --git a/external/storm-kafka/src/jvm/storm/kafka/Broker.java b/external/storm-kafka/src/jvm/storm/kafka/Broker.java new file mode 100644 index 000000000..bfa3e0b0b --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/Broker.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import java.io.Serializable; +import com.google.common.base.Objects; + +public class Broker implements Serializable, Comparable { + public final String host; + public final int port; + + public Broker(String host, int port) { + this.host = host; + this.port = port; + } + + public Broker(String host) { + this(host, 9092); + } + + @Override + public int hashCode() { + return Objects.hashCode(host, port); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final Broker other = (Broker) obj; + return Objects.equal(this.host, other.host) && Objects.equal(this.port, other.port); + } + + @Override + public String toString() { + return host + ":" + port; + } + + public static Broker fromString(String host) { + Broker hp; + String[] spec = host.split(":"); + if (spec.length == 1) { + hp = new Broker(spec[0]); + } else if (spec.length == 2) { + hp = new Broker(spec[0], Integer.parseInt(spec[1])); + } else { + throw new IllegalArgumentException("Invalid host specification: " + host); + } + return hp; + } + + + @Override + public int compareTo(Broker o) { + if (this.host.equals(o.host)) { + return this.port - o.port; + } else { + return this.host.compareTo(o.host); + } + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/BrokerHosts.java b/external/storm-kafka/src/jvm/storm/kafka/BrokerHosts.java new file mode 100644 index 000000000..fcdf0b614 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/BrokerHosts.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import java.io.Serializable; + + +public interface BrokerHosts extends Serializable { + +} \ No newline at end of file diff --git a/external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java b/external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java new file mode 100644 index 000000000..51978623f --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java @@ -0,0 +1,145 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.Config; +import backtype.storm.utils.Utils; +import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.CuratorFrameworkFactory; +import com.netflix.curator.retry.RetryNTimes; +import org.json.simple.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.trident.GlobalPartitionInformation; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.List; +import java.util.Map; + +public class DynamicBrokersReader { + + public static final Logger LOG = LoggerFactory.getLogger(DynamicBrokersReader.class); + + private CuratorFramework _curator; + private String _zkPath; + private String _topic; + + public DynamicBrokersReader(Map conf, String zkStr, String zkPath, String topic) { + _zkPath = zkPath; + _topic = topic; + try { + _curator = CuratorFrameworkFactory.newClient( + zkStr, + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)), + 15000, + new RetryNTimes(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)), + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)))); + _curator.start(); + } catch (Exception ex) { + LOG.error("can't connect to zookeeper"); + } + } + + /** + * Get all partitions with their current leaders + */ + public GlobalPartitionInformation getBrokerInfo() { + GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(); + try { + int numPartitionsForTopic = getNumPartitions(); + String brokerInfoPath = brokerPath(); + for (int partition = 0; partition < numPartitionsForTopic; partition++) { + int leader = getLeaderFor(partition); + String path = brokerInfoPath + "/" + leader; + try { + byte[] brokerData = _curator.getData().forPath(path); + Broker hp = getBrokerHost(brokerData); + globalPartitionInformation.addPartition(partition, hp); + } catch (org.apache.zookeeper.KeeperException.NoNodeException e) { + LOG.error("Node {} does not exist ", path); + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } + LOG.info("Read partition info from zookeeper: " + globalPartitionInformation); + return globalPartitionInformation; + } + + + private int getNumPartitions() { + try { + String topicBrokersPath = partitionPath(); + List children = _curator.getChildren().forPath(topicBrokersPath); + return children.size(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public String partitionPath() { + return _zkPath + "/topics/" + _topic + "/partitions"; + } + + public String brokerPath() { + return _zkPath + "/ids"; + } + + /** + * get /brokers/topics/distributedTopic/partitions/1/state + * { "controller_epoch":4, "isr":[ 1, 0 ], "leader":1, "leader_epoch":1, "version":1 } + * + * @param partition + * @return + */ + private int getLeaderFor(long partition) { + try { + String topicBrokersPath = partitionPath(); + byte[] hostPortData = _curator.getData().forPath(topicBrokersPath + "/" + partition + "/state"); + Map value = (Map) JSONValue.parse(new String(hostPortData, "UTF-8")); + Integer leader = ((Number) value.get("leader")).intValue(); + return leader; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void close() { + _curator.close(); + } + + /** + * [zk: localhost:2181(CONNECTED) 56] get /brokers/ids/0 + * { "host":"localhost", "jmx_port":9999, "port":9092, "version":1 } + * + * @param contents + * @return + */ + private Broker getBrokerHost(byte[] contents) { + try { + Map value = (Map) JSONValue.parse(new String(contents, "UTF-8")); + String host = (String) value.get("host"); + Integer port = ((Long) value.get("port")).intValue(); + return new Broker(host, port); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/DynamicPartitionConnections.java b/external/storm-kafka/src/jvm/storm/kafka/DynamicPartitionConnections.java new file mode 100644 index 000000000..f55085834 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/DynamicPartitionConnections.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import kafka.javaapi.consumer.SimpleConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.trident.IBrokerReader; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + + +public class DynamicPartitionConnections { + + public static final Logger LOG = LoggerFactory.getLogger(DynamicPartitionConnections.class); + + static class ConnectionInfo { + SimpleConsumer consumer; + Set partitions = new HashSet(); + + public ConnectionInfo(SimpleConsumer consumer) { + this.consumer = consumer; + } + } + + Map _connections = new HashMap(); + KafkaConfig _config; + IBrokerReader _reader; + + public DynamicPartitionConnections(KafkaConfig config, IBrokerReader brokerReader) { + _config = config; + _reader = brokerReader; + } + + public SimpleConsumer register(Partition partition) { + Broker broker = _reader.getCurrentBrokers().getBrokerFor(partition.partition); + return register(broker, partition.partition); + } + + public SimpleConsumer register(Broker host, int partition) { + if (!_connections.containsKey(host)) { + _connections.put(host, new ConnectionInfo(new SimpleConsumer(host.host, host.port, _config.socketTimeoutMs, _config.bufferSizeBytes, _config.clientId))); + } + ConnectionInfo info = _connections.get(host); + info.partitions.add(partition); + return info.consumer; + } + + public SimpleConsumer getConnection(Partition partition) { + ConnectionInfo info = _connections.get(partition.host); + if (info != null) { + return info.consumer; + } + return null; + } + + public void unregister(Broker port, int partition) { + ConnectionInfo info = _connections.get(port); + info.partitions.remove(partition); + if (info.partitions.isEmpty()) { + info.consumer.close(); + _connections.remove(port); + } + } + + public void unregister(Partition partition) { + unregister(partition.host, partition.partition); + } + + public void clear() { + for (ConnectionInfo info : _connections.values()) { + info.consumer.close(); + } + _connections.clear(); + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/FailedFetchException.java b/external/storm-kafka/src/jvm/storm/kafka/FailedFetchException.java new file mode 100644 index 000000000..011240e4c --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/FailedFetchException.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +public class FailedFetchException extends RuntimeException { + + public FailedFetchException(String message) { + super(message); + } + + public FailedFetchException(Exception e) { + super(e); + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java new file mode 100644 index 000000000..875629b49 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.spout.MultiScheme; +import backtype.storm.spout.RawMultiScheme; + +import java.io.Serializable; + +public class KafkaConfig implements Serializable { + + public final BrokerHosts hosts; + public final String topic; + public final String clientId; + + public int fetchSizeBytes = 1024 * 1024; + public int socketTimeoutMs = 10000; + public int bufferSizeBytes = 1024 * 1024; + public MultiScheme scheme = new RawMultiScheme(); + public boolean forceFromStart = false; + public long startOffsetTime = kafka.api.OffsetRequest.EarliestTime(); + public boolean useStartOffsetTimeIfOffsetOutOfRange = true; + public int metricsTimeBucketSizeInSecs = 60; + + public KafkaConfig(BrokerHosts hosts, String topic) { + this(hosts, topic, kafka.api.OffsetRequest.DefaultClientId()); + } + + public KafkaConfig(BrokerHosts hosts, String topic, String clientId) { + this.hosts = hosts; + this.topic = topic; + this.clientId = clientId; + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaError.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaError.java new file mode 100644 index 000000000..634af8551 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaError.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +public enum KafkaError { + NO_ERROR, + OFFSET_OUT_OF_RANGE, + INVALID_MESSAGE, + UNKNOWN_TOPIC_OR_PARTITION, + INVALID_FETCH_SIZE, + LEADER_NOT_AVAILABLE, + NOT_LEADER_FOR_PARTITION, + REQUEST_TIMED_OUT, + BROKER_NOT_AVAILABLE, + REPLICA_NOT_AVAILABLE, + MESSAGE_SIZE_TOO_LARGE, + STALE_CONTROLLER_EPOCH, + OFFSET_METADATA_TOO_LARGE, + UNKNOWN; + + public static KafkaError getError(int errorCode) { + if (errorCode < 0 || errorCode >= UNKNOWN.ordinal()) { + return UNKNOWN; + } else { + return values()[errorCode]; + } + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaSpout.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaSpout.java new file mode 100644 index 000000000..102dce193 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaSpout.java @@ -0,0 +1,190 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.Config; +import backtype.storm.metric.api.IMetric; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import kafka.message.Message; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.PartitionManager.KafkaMessageId; + +import java.util.*; + +// TODO: need to add blacklisting +// TODO: need to make a best effort to not re-emit messages if don't have to +public class KafkaSpout extends BaseRichSpout { + public static class MessageAndRealOffset { + public Message msg; + public long offset; + + public MessageAndRealOffset(Message msg, long offset) { + this.msg = msg; + this.offset = offset; + } + } + + static enum EmitState { + EMITTED_MORE_LEFT, + EMITTED_END, + NO_EMITTED + } + + public static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class); + + String _uuid = UUID.randomUUID().toString(); + SpoutConfig _spoutConfig; + SpoutOutputCollector _collector; + PartitionCoordinator _coordinator; + DynamicPartitionConnections _connections; + ZkState _state; + + long _lastUpdateMs = 0; + + int _currPartitionIndex = 0; + + public KafkaSpout(SpoutConfig spoutConf) { + _spoutConfig = spoutConf; + } + + @Override + public void open(Map conf, final TopologyContext context, final SpoutOutputCollector collector) { + _collector = collector; + + Map stateConf = new HashMap(conf); + List zkServers = _spoutConfig.zkServers; + if (zkServers == null) { + zkServers = (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS); + } + Integer zkPort = _spoutConfig.zkPort; + if (zkPort == null) { + zkPort = ((Number) conf.get(Config.STORM_ZOOKEEPER_PORT)).intValue(); + } + stateConf.put(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, zkServers); + stateConf.put(Config.TRANSACTIONAL_ZOOKEEPER_PORT, zkPort); + stateConf.put(Config.TRANSACTIONAL_ZOOKEEPER_ROOT, _spoutConfig.zkRoot); + _state = new ZkState(stateConf); + + _connections = new DynamicPartitionConnections(_spoutConfig, KafkaUtils.makeBrokerReader(conf, _spoutConfig)); + + // using TransactionalState like this is a hack + int totalTasks = context.getComponentTasks(context.getThisComponentId()).size(); + if (_spoutConfig.hosts instanceof StaticHosts) { + _coordinator = new StaticCoordinator(_connections, conf, _spoutConfig, _state, context.getThisTaskIndex(), totalTasks, _uuid); + } else { + _coordinator = new ZkCoordinator(_connections, conf, _spoutConfig, _state, context.getThisTaskIndex(), totalTasks, _uuid); + } + + context.registerMetric("kafkaOffset", new IMetric() { + KafkaUtils.KafkaOffsetMetric _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_spoutConfig.topic, _connections); + + @Override + public Object getValueAndReset() { + List pms = _coordinator.getMyManagedPartitions(); + Set latestPartitions = new HashSet(); + for (PartitionManager pm : pms) { + latestPartitions.add(pm.getPartition()); + } + _kafkaOffsetMetric.refreshPartitions(latestPartitions); + for (PartitionManager pm : pms) { + _kafkaOffsetMetric.setLatestEmittedOffset(pm.getPartition(), pm.lastCompletedOffset()); + } + return _kafkaOffsetMetric.getValueAndReset(); + } + }, _spoutConfig.metricsTimeBucketSizeInSecs); + + context.registerMetric("kafkaPartition", new IMetric() { + @Override + public Object getValueAndReset() { + List pms = _coordinator.getMyManagedPartitions(); + Map concatMetricsDataMaps = new HashMap(); + for (PartitionManager pm : pms) { + concatMetricsDataMaps.putAll(pm.getMetricsDataMap()); + } + return concatMetricsDataMaps; + } + }, _spoutConfig.metricsTimeBucketSizeInSecs); + } + + @Override + public void close() { + _state.close(); + } + + @Override + public void nextTuple() { + List managers = _coordinator.getMyManagedPartitions(); + for (int i = 0; i < managers.size(); i++) { + + // in case the number of managers decreased + _currPartitionIndex = _currPartitionIndex % managers.size(); + EmitState state = managers.get(_currPartitionIndex).next(_collector); + if (state != EmitState.EMITTED_MORE_LEFT) { + _currPartitionIndex = (_currPartitionIndex + 1) % managers.size(); + } + if (state != EmitState.NO_EMITTED) { + break; + } + } + + long now = System.currentTimeMillis(); + if ((now - _lastUpdateMs) > _spoutConfig.stateUpdateIntervalMs) { + commit(); + } + } + + @Override + public void ack(Object msgId) { + KafkaMessageId id = (KafkaMessageId) msgId; + PartitionManager m = _coordinator.getManager(id.partition); + if (m != null) { + m.ack(id.offset); + } + } + + @Override + public void fail(Object msgId) { + KafkaMessageId id = (KafkaMessageId) msgId; + PartitionManager m = _coordinator.getManager(id.partition); + if (m != null) { + m.fail(id.offset); + } + } + + @Override + public void deactivate() { + commit(); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(_spoutConfig.scheme.getOutputFields()); + } + + private void commit() { + _lastUpdateMs = System.currentTimeMillis(); + for (PartitionManager manager : _coordinator.getMyManagedPartitions()) { + manager.commit(); + } + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java new file mode 100644 index 000000000..3d355e56a --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java @@ -0,0 +1,235 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.metric.api.IMetric; +import backtype.storm.utils.Utils; +import com.google.common.base.Preconditions; +import kafka.api.FetchRequest; +import kafka.api.FetchRequestBuilder; +import kafka.api.PartitionOffsetRequestInfo; +import kafka.common.TopicAndPartition; +import kafka.javaapi.FetchResponse; +import kafka.javaapi.OffsetRequest; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.message.Message; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.trident.GlobalPartitionInformation; +import storm.kafka.trident.IBrokerReader; +import storm.kafka.trident.StaticBrokerReader; +import storm.kafka.trident.ZkBrokerReader; + +import java.net.ConnectException; +import java.nio.ByteBuffer; +import java.util.*; + + +public class KafkaUtils { + + public static final Logger LOG = LoggerFactory.getLogger(KafkaUtils.class); + private static final int NO_OFFSET = -5; + + + public static IBrokerReader makeBrokerReader(Map stormConf, KafkaConfig conf) { + if (conf.hosts instanceof StaticHosts) { + return new StaticBrokerReader(((StaticHosts) conf.hosts).getPartitionInformation()); + } else { + return new ZkBrokerReader(stormConf, conf.topic, (ZkHosts) conf.hosts); + } + } + + + public static long getOffset(SimpleConsumer consumer, String topic, int partition, KafkaConfig config) { + long startOffsetTime = kafka.api.OffsetRequest.LatestTime(); + if ( config.forceFromStart ) { + startOffsetTime = config.startOffsetTime; + } + return getOffset(consumer, topic, partition, startOffsetTime); + } + + public static long getOffset(SimpleConsumer consumer, String topic, int partition, long startOffsetTime) { + TopicAndPartition topicAndPartition = new TopicAndPartition(topic, partition); + Map requestInfo = new HashMap(); + requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(startOffsetTime, 1)); + OffsetRequest request = new OffsetRequest( + requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId()); + + long[] offsets = consumer.getOffsetsBefore(request).offsets(topic, partition); + if (offsets.length > 0) { + return offsets[0]; + } else { + return NO_OFFSET; + } + } + + public static class KafkaOffsetMetric implements IMetric { + Map _partitionToOffset = new HashMap(); + Set _partitions; + String _topic; + DynamicPartitionConnections _connections; + + public KafkaOffsetMetric(String topic, DynamicPartitionConnections connections) { + _topic = topic; + _connections = connections; + } + + public void setLatestEmittedOffset(Partition partition, long offset) { + _partitionToOffset.put(partition, offset); + } + + @Override + public Object getValueAndReset() { + try { + long totalSpoutLag = 0; + long totalEarliestTimeOffset = 0; + long totalLatestTimeOffset = 0; + long totalLatestEmittedOffset = 0; + HashMap ret = new HashMap(); + if (_partitions != null && _partitions.size() == _partitionToOffset.size()) { + for (Map.Entry e : _partitionToOffset.entrySet()) { + Partition partition = e.getKey(); + SimpleConsumer consumer = _connections.getConnection(partition); + if (consumer == null) { + LOG.warn("partitionToOffset contains partition not found in _connections. Stale partition data?"); + return null; + } + long earliestTimeOffset = getOffset(consumer, _topic, partition.partition, kafka.api.OffsetRequest.EarliestTime()); + long latestTimeOffset = getOffset(consumer, _topic, partition.partition, kafka.api.OffsetRequest.LatestTime()); + if (earliestTimeOffset == 0 || latestTimeOffset == 0) { + LOG.warn("No data found in Kafka Partition " + partition.getId()); + return null; + } + long latestEmittedOffset = e.getValue(); + long spoutLag = latestTimeOffset - latestEmittedOffset; + ret.put(partition.getId() + "/" + "spoutLag", spoutLag); + ret.put(partition.getId() + "/" + "earliestTimeOffset", earliestTimeOffset); + ret.put(partition.getId() + "/" + "latestTimeOffset", latestTimeOffset); + ret.put(partition.getId() + "/" + "latestEmittedOffset", latestEmittedOffset); + totalSpoutLag += spoutLag; + totalEarliestTimeOffset += earliestTimeOffset; + totalLatestTimeOffset += latestTimeOffset; + totalLatestEmittedOffset += latestEmittedOffset; + } + ret.put("totalSpoutLag", totalSpoutLag); + ret.put("totalEarliestTimeOffset", totalEarliestTimeOffset); + ret.put("totalLatestTimeOffset", totalLatestTimeOffset); + ret.put("totalLatestEmittedOffset", totalLatestEmittedOffset); + return ret; + } else { + LOG.info("Metrics Tick: Not enough data to calculate spout lag."); + } + } catch (Throwable t) { + LOG.warn("Metrics Tick: Exception when computing kafkaOffset metric.", t); + } + return null; + } + + public void refreshPartitions(Set partitions) { + _partitions = partitions; + Iterator it = _partitionToOffset.keySet().iterator(); + while (it.hasNext()) { + if (!partitions.contains(it.next())) { + it.remove(); + } + } + } + } + + public static ByteBufferMessageSet fetchMessages(KafkaConfig config, SimpleConsumer consumer, Partition partition, long offset) { + ByteBufferMessageSet msgs = null; + String topic = config.topic; + int partitionId = partition.partition; + for (int errors = 0; errors < 2 && msgs == null; errors++) { + FetchRequestBuilder builder = new FetchRequestBuilder(); + FetchRequest fetchRequest = builder.addFetch(topic, partitionId, offset, config.fetchSizeBytes). + clientId(config.clientId).build(); + FetchResponse fetchResponse; + try { + fetchResponse = consumer.fetch(fetchRequest); + } catch (Exception e) { + if (e instanceof ConnectException) { + throw new FailedFetchException(e); + } else { + throw new RuntimeException(e); + } + } + if (fetchResponse.hasError()) { + KafkaError error = KafkaError.getError(fetchResponse.errorCode(topic, partitionId)); + if (error.equals(KafkaError.OFFSET_OUT_OF_RANGE) && config.useStartOffsetTimeIfOffsetOutOfRange && errors == 0) { + long startOffset = getOffset(consumer, topic, partitionId, config.startOffsetTime); + LOG.warn("Got fetch request with offset out of range: [" + offset + "]; " + + "retrying with default start offset time from configuration. " + + "configured start offset time: [" + config.startOffsetTime + "] offset: [" + startOffset + "]"); + offset = startOffset; + } else { + String message = "Error fetching data from [" + partition + "] for topic [" + topic + "]: [" + error + "]"; + LOG.error(message); + throw new FailedFetchException(message); + } + } else { + msgs = fetchResponse.messageSet(topic, partitionId); + } + } + return msgs; + } + + + public static Iterable> generateTuples(KafkaConfig kafkaConfig, Message msg) { + Iterable> tups; + ByteBuffer payload = msg.payload(); + ByteBuffer key = msg.key(); + if (key != null && kafkaConfig.scheme instanceof KeyValueSchemeAsMultiScheme) { + tups = ((KeyValueSchemeAsMultiScheme) kafkaConfig.scheme).deserializeKeyAndValue(Utils.toByteArray(key), Utils.toByteArray(payload)); + } else { + tups = kafkaConfig.scheme.deserialize(Utils.toByteArray(payload)); + } + return tups; + } + + + public static List calculatePartitionsForTask(GlobalPartitionInformation partitionInformation, int totalTasks, int taskIndex) { + Preconditions.checkArgument(taskIndex < totalTasks, "task index must be less that total tasks"); + List partitions = partitionInformation.getOrderedPartitions(); + int numPartitions = partitions.size(); + if (numPartitions < totalTasks) { + LOG.warn("there are more tasks than partitions (tasks: " + totalTasks + "; partitions: " + numPartitions + "), some tasks will be idle"); + } + List taskPartitions = new ArrayList(); + for (int i = taskIndex; i < numPartitions; i += totalTasks) { + Partition taskPartition = partitions.get(i); + taskPartitions.add(taskPartition); + } + logPartitionMapping(totalTasks, taskIndex, taskPartitions); + return taskPartitions; + } + + private static void logPartitionMapping(int totalTasks, int taskIndex, List taskPartitions) { + String taskPrefix = taskId(taskIndex, totalTasks); + if (taskPartitions.isEmpty()) { + LOG.warn(taskPrefix + "no partitions assigned"); + } else { + LOG.info(taskPrefix + "assigned " + taskPartitions); + } + } + + public static String taskId(int taskIndex, int totalTasks) { + return "Task [" + (taskIndex + 1) + "/" + totalTasks + "] "; + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java b/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java new file mode 100644 index 000000000..f42f7c82d --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.spout.Scheme; + +import java.util.List; + +public interface KeyValueScheme extends Scheme { + + public List deserializeKeyAndValue(byte[] key, byte[] value); + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/KeyValueSchemeAsMultiScheme.java b/external/storm-kafka/src/jvm/storm/kafka/KeyValueSchemeAsMultiScheme.java new file mode 100644 index 000000000..a570e7db8 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/KeyValueSchemeAsMultiScheme.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.spout.SchemeAsMultiScheme; +import java.util.Arrays; +import java.util.List; + +public class KeyValueSchemeAsMultiScheme extends SchemeAsMultiScheme{ + + public KeyValueSchemeAsMultiScheme(KeyValueScheme scheme) { + super(scheme); + } + + public Iterable> deserializeKeyAndValue(final byte[] key, final byte[] value) { + List o = ((KeyValueScheme)scheme).deserializeKeyAndValue(key, value); + if(o == null) return null; + else return Arrays.asList(o); + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/Partition.java b/external/storm-kafka/src/jvm/storm/kafka/Partition.java new file mode 100644 index 000000000..fe6ab4f96 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/Partition.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import com.google.common.base.Objects; +import storm.trident.spout.ISpoutPartition; + + +public class Partition implements ISpoutPartition { + + public final Broker host; + public final int partition; + + public Partition(Broker host, int partition) { + this.host = host; + this.partition = partition; + } + + @Override + public int hashCode() { + return Objects.hashCode(host, partition); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final Partition other = (Partition) obj; + return Objects.equal(this.host, other.host) && Objects.equal(this.partition, other.partition); + } + + @Override + public String toString() { + return "Partition{" + + "host=" + host + + ", partition=" + partition + + '}'; + } + + @Override + public String getId() { + return "partition_" + partition; + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/PartitionCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/PartitionCoordinator.java new file mode 100644 index 000000000..60cc237bd --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/PartitionCoordinator.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import java.util.List; + +public interface PartitionCoordinator { + List getMyManagedPartitions(); + + PartitionManager getManager(Partition partition); +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java new file mode 100644 index 000000000..950442710 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java @@ -0,0 +1,241 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.Config; +import backtype.storm.metric.api.CombinedMetric; +import backtype.storm.metric.api.CountMetric; +import backtype.storm.metric.api.MeanReducer; +import backtype.storm.metric.api.ReducedMetric; +import backtype.storm.spout.SpoutOutputCollector; +import com.google.common.collect.ImmutableMap; +import kafka.api.OffsetRequest; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.message.MessageAndOffset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.KafkaSpout.EmitState; +import storm.kafka.KafkaSpout.MessageAndRealOffset; +import storm.kafka.trident.MaxMetric; + +import java.util.*; + +public class PartitionManager { + public static final Logger LOG = LoggerFactory.getLogger(PartitionManager.class); + private final CombinedMetric _fetchAPILatencyMax; + private final ReducedMetric _fetchAPILatencyMean; + private final CountMetric _fetchAPICallCount; + private final CountMetric _fetchAPIMessageCount; + + static class KafkaMessageId { + public Partition partition; + public long offset; + + public KafkaMessageId(Partition partition, long offset) { + this.partition = partition; + this.offset = offset; + } + } + + Long _emittedToOffset; + SortedSet _pending = new TreeSet(); + Long _committedTo; + LinkedList _waitingToEmit = new LinkedList(); + Partition _partition; + SpoutConfig _spoutConfig; + String _topologyInstanceId; + SimpleConsumer _consumer; + DynamicPartitionConnections _connections; + ZkState _state; + Map _stormConf; + + + public PartitionManager(DynamicPartitionConnections connections, String topologyInstanceId, ZkState state, Map stormConf, SpoutConfig spoutConfig, Partition id) { + _partition = id; + _connections = connections; + _spoutConfig = spoutConfig; + _topologyInstanceId = topologyInstanceId; + _consumer = connections.register(id.host, id.partition); + _state = state; + _stormConf = stormConf; + + String jsonTopologyId = null; + Long jsonOffset = null; + String path = committedPath(); + try { + Map json = _state.readJSON(path); + LOG.info("Read partition information from: " + path + " --> " + json ); + if (json != null) { + jsonTopologyId = (String) ((Map) json.get("topology")).get("id"); + jsonOffset = (Long) json.get("offset"); + } + } catch (Throwable e) { + LOG.warn("Error reading and/or parsing at ZkNode: " + path, e); + } + + if (jsonTopologyId == null || jsonOffset == null) { // failed to parse JSON? + _committedTo = KafkaUtils.getOffset(_consumer, spoutConfig.topic, id.partition, spoutConfig); + LOG.info("No partition information found, using configuration to determine offset"); + } else if (!topologyInstanceId.equals(jsonTopologyId) && spoutConfig.forceFromStart) { + _committedTo = KafkaUtils.getOffset(_consumer, spoutConfig.topic, id.partition, spoutConfig.startOffsetTime); + LOG.info("Topology change detected and reset from start forced, using configuration to determine offset"); + } else { + _committedTo = jsonOffset; + LOG.info("Read last commit offset from zookeeper: " + _committedTo + "; old topology_id: " + jsonTopologyId + " - new topology_id: " + topologyInstanceId ); + } + + LOG.info("Starting " + _partition + " from offset " + _committedTo); + _emittedToOffset = _committedTo; + + _fetchAPILatencyMax = new CombinedMetric(new MaxMetric()); + _fetchAPILatencyMean = new ReducedMetric(new MeanReducer()); + _fetchAPICallCount = new CountMetric(); + _fetchAPIMessageCount = new CountMetric(); + } + + public Map getMetricsDataMap() { + Map ret = new HashMap(); + ret.put(_partition + "/fetchAPILatencyMax", _fetchAPILatencyMax.getValueAndReset()); + ret.put(_partition + "/fetchAPILatencyMean", _fetchAPILatencyMean.getValueAndReset()); + ret.put(_partition + "/fetchAPICallCount", _fetchAPICallCount.getValueAndReset()); + ret.put(_partition + "/fetchAPIMessageCount", _fetchAPIMessageCount.getValueAndReset()); + return ret; + } + + //returns false if it's reached the end of current batch + public EmitState next(SpoutOutputCollector collector) { + if (_waitingToEmit.isEmpty()) { + fill(); + } + while (true) { + MessageAndRealOffset toEmit = _waitingToEmit.pollFirst(); + if (toEmit == null) { + return EmitState.NO_EMITTED; + } + Iterable> tups = KafkaUtils.generateTuples(_spoutConfig, toEmit.msg); + if (tups != null) { + for (List tup : tups) { + collector.emit(tup, new KafkaMessageId(_partition, toEmit.offset)); + } + break; + } else { + ack(toEmit.offset); + } + } + if (!_waitingToEmit.isEmpty()) { + return EmitState.EMITTED_MORE_LEFT; + } else { + return EmitState.EMITTED_END; + } + } + + private void fill() { + long start = System.nanoTime(); + ByteBufferMessageSet msgs = KafkaUtils.fetchMessages(_spoutConfig, _consumer, _partition, _emittedToOffset); + long end = System.nanoTime(); + long millis = (end - start) / 1000000; + _fetchAPILatencyMax.update(millis); + _fetchAPILatencyMean.update(millis); + _fetchAPICallCount.incr(); + int numMessages = countMessages(msgs); + _fetchAPIMessageCount.incrBy(numMessages); + + if (numMessages > 0) { + LOG.info("Fetched " + numMessages + " messages from: " + _partition); + } + for (MessageAndOffset msg : msgs) { + _pending.add(_emittedToOffset); + _waitingToEmit.add(new MessageAndRealOffset(msg.message(), _emittedToOffset)); + _emittedToOffset = msg.nextOffset(); + } + if (numMessages > 0) { + LOG.info("Added " + numMessages + " messages from: " + _partition + " to internal buffers"); + } + } + + private int countMessages(ByteBufferMessageSet messageSet) { + int counter = 0; + for (MessageAndOffset messageAndOffset : messageSet) { + counter = counter + 1; + } + return counter; + } + + public void ack(Long offset) { + _pending.remove(offset); + } + + public void fail(Long offset) { + //TODO: should it use in-memory ack set to skip anything that's been acked but not committed??? + // things might get crazy with lots of timeouts + if (_emittedToOffset > offset) { + _emittedToOffset = offset; + _pending.tailSet(offset).clear(); + } + } + + public void commit() { + long lastCompletedOffset = lastCompletedOffset(); + if (lastCompletedOffset != lastCommittedOffset()) { + LOG.info("Writing last completed offset (" + lastCompletedOffset + ") to ZK for " + _partition + " for topology: " + _topologyInstanceId); + Map data = ImmutableMap.builder() + .put("topology", ImmutableMap.of("id", _topologyInstanceId, + "name", _stormConf.get(Config.TOPOLOGY_NAME))) + .put("offset", lastCompletedOffset) + .put("partition", _partition.partition) + .put("broker", ImmutableMap.of("host", _partition.host.host, + "port", _partition.host.port)) + .put("topic", _spoutConfig.topic).build(); + _state.writeJSON(committedPath(), data); + _committedTo = lastCompletedOffset; + LOG.info("Wrote last completed offset (" + lastCompletedOffset + ") to ZK for " + _partition + " for topology: " + _topologyInstanceId); + } else { + LOG.info("No new offset for " + _partition + " for topology: " + _topologyInstanceId); + } + } + + private String committedPath() { + return _spoutConfig.zkRoot + "/" + _spoutConfig.id + "/" + _spoutConfig.topic + "/" + _partition.getId(); + } + + public long queryPartitionOffsetLatestTime() { + return KafkaUtils.getOffset(_consumer, _spoutConfig.topic, _partition.partition, + OffsetRequest.LatestTime()); + } + + public long lastCommittedOffset() { + return _committedTo; + } + + public long lastCompletedOffset() { + if (_pending.isEmpty()) { + return _emittedToOffset; + } else { + return _pending.first(); + } + } + + public Partition getPartition() { + return _partition; + } + + public void close() { + _connections.unregister(_partition.host, _partition.partition); + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/SpoutConfig.java b/external/storm-kafka/src/jvm/storm/kafka/SpoutConfig.java new file mode 100644 index 000000000..1b66026e1 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/SpoutConfig.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import java.io.Serializable; +import java.util.List; + + +public class SpoutConfig extends KafkaConfig implements Serializable { + public List zkServers = null; + public Integer zkPort = null; + public String zkRoot = null; + public String id = null; + public long stateUpdateIntervalMs = 2000; + + public SpoutConfig(BrokerHosts hosts, String topic, String zkRoot, String id) { + super(hosts, topic); + this.zkRoot = zkRoot; + this.id = id; + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/StaticCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/StaticCoordinator.java new file mode 100644 index 000000000..456a2a134 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/StaticCoordinator.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +public class StaticCoordinator implements PartitionCoordinator { + Map _managers = new HashMap(); + List _allManagers = new ArrayList(); + + public StaticCoordinator(DynamicPartitionConnections connections, Map stormConf, SpoutConfig config, ZkState state, int taskIndex, int totalTasks, String topologyInstanceId) { + StaticHosts hosts = (StaticHosts) config.hosts; + List myPartitions = KafkaUtils.calculatePartitionsForTask(hosts.getPartitionInformation(), totalTasks, taskIndex); + for (Partition myPartition : myPartitions) { + _managers.put(myPartition, new PartitionManager(connections, topologyInstanceId, state, stormConf, config, myPartition)); + } + _allManagers = new ArrayList(_managers.values()); + } + + @Override + public List getMyManagedPartitions() { + return _allManagers; + } + + public PartitionManager getManager(Partition partition) { + return _managers.get(partition); + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/StaticHosts.java b/external/storm-kafka/src/jvm/storm/kafka/StaticHosts.java new file mode 100644 index 000000000..bee71187f --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/StaticHosts.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import storm.kafka.trident.GlobalPartitionInformation; + +/** + * Date: 11/05/2013 + * Time: 14:43 + */ +public class StaticHosts implements BrokerHosts { + + + private GlobalPartitionInformation partitionInformation; + + public StaticHosts(GlobalPartitionInformation partitionInformation) { + this.partitionInformation = partitionInformation; + } + + public GlobalPartitionInformation getPartitionInformation() { + return partitionInformation; + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/StaticPartitionConnections.java b/external/storm-kafka/src/jvm/storm/kafka/StaticPartitionConnections.java new file mode 100644 index 000000000..1353b6c05 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/StaticPartitionConnections.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import kafka.javaapi.consumer.SimpleConsumer; + +import java.util.HashMap; +import java.util.Map; + +public class StaticPartitionConnections { + Map _kafka = new HashMap(); + KafkaConfig _config; + StaticHosts hosts; + + public StaticPartitionConnections(KafkaConfig conf) { + _config = conf; + if (!(conf.hosts instanceof StaticHosts)) { + throw new RuntimeException("Must configure with static hosts"); + } + this.hosts = (StaticHosts) conf.hosts; + } + + public SimpleConsumer getConsumer(int partition) { + if (!_kafka.containsKey(partition)) { + Broker hp = hosts.getPartitionInformation().getBrokerFor(partition); + _kafka.put(partition, new SimpleConsumer(hp.host, hp.port, _config.socketTimeoutMs, _config.bufferSizeBytes, _config.clientId)); + + } + return _kafka.get(partition); + } + + public void close() { + for (SimpleConsumer consumer : _kafka.values()) { + consumer.close(); + } + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/StringKeyValueScheme.java b/external/storm-kafka/src/jvm/storm/kafka/StringKeyValueScheme.java new file mode 100644 index 000000000..41cacb614 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/StringKeyValueScheme.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.tuple.Values; +import com.google.common.collect.ImmutableMap; + +import java.util.List; + +public class StringKeyValueScheme extends StringScheme implements KeyValueScheme { + + @Override + public List deserializeKeyAndValue(byte[] key, byte[] value) { + if ( key == null ) { + return deserialize(value); + } + String keyString = StringScheme.deserializeString(key); + String valueString = StringScheme.deserializeString(value); + return new Values(ImmutableMap.of(keyString, valueString)); + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/StringScheme.java b/external/storm-kafka/src/jvm/storm/kafka/StringScheme.java new file mode 100644 index 000000000..102ea6958 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/StringScheme.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.spout.Scheme; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +import java.io.UnsupportedEncodingException; +import java.util.List; + +public class StringScheme implements Scheme { + + public static final String STRING_SCHEME_KEY = "str"; + + public List deserialize(byte[] bytes) { + return new Values(deserializeString(bytes)); + } + + public static String deserializeString(byte[] string) { + try { + return new String(string, "UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + public Fields getOutputFields() { + return new Fields(STRING_SCHEME_KEY); + } +} \ No newline at end of file diff --git a/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java new file mode 100644 index 000000000..e414d063a --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.trident.GlobalPartitionInformation; + +import java.util.*; + +import static storm.kafka.KafkaUtils.taskId; + +public class ZkCoordinator implements PartitionCoordinator { + public static final Logger LOG = LoggerFactory.getLogger(ZkCoordinator.class); + + SpoutConfig _spoutConfig; + int _taskIndex; + int _totalTasks; + String _topologyInstanceId; + Map _managers = new HashMap(); + List _cachedList; + Long _lastRefreshTime = null; + int _refreshFreqMs; + DynamicPartitionConnections _connections; + DynamicBrokersReader _reader; + ZkState _state; + Map _stormConf; + + public ZkCoordinator(DynamicPartitionConnections connections, Map stormConf, SpoutConfig spoutConfig, ZkState state, int taskIndex, int totalTasks, String topologyInstanceId) { + this(connections, stormConf, spoutConfig, state, taskIndex, totalTasks, topologyInstanceId, buildReader(stormConf, spoutConfig)); + } + + public ZkCoordinator(DynamicPartitionConnections connections, Map stormConf, SpoutConfig spoutConfig, ZkState state, int taskIndex, int totalTasks, String topologyInstanceId, DynamicBrokersReader reader) { + _spoutConfig = spoutConfig; + _connections = connections; + _taskIndex = taskIndex; + _totalTasks = totalTasks; + _topologyInstanceId = topologyInstanceId; + _stormConf = stormConf; + _state = state; + ZkHosts brokerConf = (ZkHosts) spoutConfig.hosts; + _refreshFreqMs = brokerConf.refreshFreqSecs * 1000; + _reader = reader; + } + + private static DynamicBrokersReader buildReader(Map stormConf, SpoutConfig spoutConfig) { + ZkHosts hosts = (ZkHosts) spoutConfig.hosts; + return new DynamicBrokersReader(stormConf, hosts.brokerZkStr, hosts.brokerZkPath, spoutConfig.topic); + } + + @Override + public List getMyManagedPartitions() { + if (_lastRefreshTime == null || (System.currentTimeMillis() - _lastRefreshTime) > _refreshFreqMs) { + refresh(); + _lastRefreshTime = System.currentTimeMillis(); + } + return _cachedList; + } + + void refresh() { + try { + LOG.info(taskId(_taskIndex, _totalTasks) + "Refreshing partition manager connections"); + GlobalPartitionInformation brokerInfo = _reader.getBrokerInfo(); + List mine = KafkaUtils.calculatePartitionsForTask(brokerInfo, _totalTasks, _taskIndex); + + Set curr = _managers.keySet(); + Set newPartitions = new HashSet(mine); + newPartitions.removeAll(curr); + + Set deletedPartitions = new HashSet(curr); + deletedPartitions.removeAll(mine); + + LOG.info(taskId(_taskIndex, _totalTasks) + "Deleted partition managers: " + deletedPartitions.toString()); + + for (Partition id : deletedPartitions) { + PartitionManager man = _managers.remove(id); + man.close(); + } + LOG.info(taskId(_taskIndex, _totalTasks) + "New partition managers: " + newPartitions.toString()); + + for (Partition id : newPartitions) { + PartitionManager man = new PartitionManager(_connections, _topologyInstanceId, _state, _stormConf, _spoutConfig, id); + _managers.put(id, man); + } + + } catch (Exception e) { + throw new RuntimeException(e); + } + _cachedList = new ArrayList(_managers.values()); + LOG.info(taskId(_taskIndex, _totalTasks) + "Finished refreshing"); + } + + @Override + public PartitionManager getManager(Partition partition) { + return _managers.get(partition); + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/ZkHosts.java b/external/storm-kafka/src/jvm/storm/kafka/ZkHosts.java new file mode 100644 index 000000000..d9acc6626 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/ZkHosts.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + + +public class ZkHosts implements BrokerHosts { + private static final String DEFAULT_ZK_PATH = "/brokers"; + + public String brokerZkStr = null; + public String brokerZkPath = null; // e.g., /kafka/brokers + public int refreshFreqSecs = 60; + + public ZkHosts(String brokerZkStr, String brokerZkPath) { + this.brokerZkStr = brokerZkStr; + this.brokerZkPath = brokerZkPath; + } + + public ZkHosts(String brokerZkStr) { + this(brokerZkStr, DEFAULT_ZK_PATH); + } +} \ No newline at end of file diff --git a/external/storm-kafka/src/jvm/storm/kafka/ZkState.java b/external/storm-kafka/src/jvm/storm/kafka/ZkState.java new file mode 100644 index 000000000..52585efd1 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/ZkState.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka; + +import backtype.storm.Config; +import backtype.storm.utils.Utils; +import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.CuratorFrameworkFactory; +import com.netflix.curator.retry.RetryNTimes; +import org.apache.zookeeper.CreateMode; +import org.json.simple.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ZkState { + public static final Logger LOG = LoggerFactory.getLogger(ZkState.class); + CuratorFramework _curator; + + private CuratorFramework newCurator(Map stateConf) throws Exception { + Integer port = (Integer) stateConf.get(Config.TRANSACTIONAL_ZOOKEEPER_PORT); + String serverPorts = ""; + for (String server : (List) stateConf.get(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS)) { + serverPorts = serverPorts + server + ":" + port + ","; + } + return CuratorFrameworkFactory.newClient(serverPorts, + Utils.getInt(stateConf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)), + 15000, + new RetryNTimes(Utils.getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)), + Utils.getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)))); + } + + public CuratorFramework getCurator() { + assert _curator != null; + return _curator; + } + + public ZkState(Map stateConf) { + stateConf = new HashMap(stateConf); + + try { + _curator = newCurator(stateConf); + _curator.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void writeJSON(String path, Map data) { + LOG.info("Writing " + path + " the data " + data.toString()); + writeBytes(path, JSONValue.toJSONString(data).getBytes(Charset.forName("UTF-8"))); + } + + public void writeBytes(String path, byte[] bytes) { + try { + if (_curator.checkExists().forPath(path) == null) { + _curator.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(path, bytes); + } else { + _curator.setData().forPath(path, bytes); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public Map readJSON(String path) { + try { + byte[] b = readBytes(path); + if (b == null) { + return null; + } + return (Map) JSONValue.parse(new String(b, "UTF-8")); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public byte[] readBytes(String path) { + try { + if (_curator.checkExists().forPath(path) != null) { + return _curator.getData().forPath(path); + } else { + return null; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void close() { + _curator.close(); + _curator = null; + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java new file mode 100644 index 000000000..b9ea94859 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.bolt; + +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Tuple; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Properties; + + +/** + * Bolt implementation that can send Tuple data to Kafka + *

+ * It expects the producer configuration and topic in storm config under + *

+ * 'kafka.broker.properties' and 'topic' + *

+ * respectively. + */ +public class KafkaBolt extends BaseRichBolt { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaBolt.class); + + public static final String TOPIC = "topic"; + public static final String KAFKA_BROKER_PROPERTIES = "kafka.broker.properties"; + + public static final String BOLT_KEY = "key"; + public static final String BOLT_MESSAGE = "message"; + + private Producer producer; + private OutputCollector collector; + private String topic; + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + Map configMap = (Map) stormConf.get(KAFKA_BROKER_PROPERTIES); + Properties properties = new Properties(); + properties.putAll(configMap); + ProducerConfig config = new ProducerConfig(properties); + producer = new Producer(config); + this.topic = (String) stormConf.get(TOPIC); + this.collector = collector; + } + + @Override + public void execute(Tuple input) { + K key = null; + if (input.contains(BOLT_KEY)) { + key = (K) input.getValueByField(BOLT_KEY); + } + V message = (V) input.getValueByField(BOLT_MESSAGE); + try { + producer.send(new KeyedMessage(topic, key, message)); + } catch (Exception ex) { + LOG.error("Could not send message with key '" + key + "' and value '" + message + "'", ex); + } finally { + collector.ack(input); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/Coordinator.java b/external/storm-kafka/src/jvm/storm/kafka/trident/Coordinator.java new file mode 100644 index 000000000..c395f8ca8 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/Coordinator.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import storm.kafka.KafkaUtils; +import storm.trident.spout.IOpaquePartitionedTridentSpout; +import storm.trident.spout.IPartitionedTridentSpout; + +import java.util.Map; + +class Coordinator implements IPartitionedTridentSpout.Coordinator, IOpaquePartitionedTridentSpout.Coordinator { + + private IBrokerReader reader; + private TridentKafkaConfig config; + + public Coordinator(Map conf, TridentKafkaConfig tridentKafkaConfig) { + config = tridentKafkaConfig; + reader = KafkaUtils.makeBrokerReader(conf, config); + } + + @Override + public void close() { + config.coordinator.close(); + } + + @Override + public boolean isReady(long txid) { + return config.coordinator.isReady(txid); + } + + @Override + public GlobalPartitionInformation getPartitionsForBatch() { + return reader.getCurrentBrokers(); + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/DefaultCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/trident/DefaultCoordinator.java new file mode 100644 index 000000000..04e1396b7 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/DefaultCoordinator.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +public class DefaultCoordinator implements IBatchCoordinator { + + @Override + public boolean isReady(long txid) { + return true; + } + + @Override + public void close() { + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/GlobalPartitionInformation.java b/external/storm-kafka/src/jvm/storm/kafka/trident/GlobalPartitionInformation.java new file mode 100644 index 000000000..ae136e569 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/GlobalPartitionInformation.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import storm.kafka.Broker; +import storm.kafka.Partition; + +import java.io.Serializable; +import java.util.*; + +import com.google.common.base.Objects; + + +public class GlobalPartitionInformation implements Iterable, Serializable { + + private Map partitionMap; + + public GlobalPartitionInformation() { + partitionMap = new TreeMap(); + } + + public void addPartition(int partitionId, Broker broker) { + partitionMap.put(partitionId, broker); + } + + @Override + public String toString() { + return "GlobalPartitionInformation{" + + "partitionMap=" + partitionMap + + '}'; + } + + public Broker getBrokerFor(Integer partitionId) { + return partitionMap.get(partitionId); + } + + public List getOrderedPartitions() { + List partitions = new LinkedList(); + for (Map.Entry partition : partitionMap.entrySet()) { + partitions.add(new Partition(partition.getValue(), partition.getKey())); + } + return partitions; + } + + @Override + public Iterator iterator() { + final Iterator> iterator = partitionMap.entrySet().iterator(); + + return new Iterator() { + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Partition next() { + Map.Entry next = iterator.next(); + return new Partition(next.getValue(), next.getKey()); + } + + @Override + public void remove() { + iterator.remove(); + } + }; + } + + @Override + public int hashCode() { + return Objects.hashCode(partitionMap); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final GlobalPartitionInformation other = (GlobalPartitionInformation) obj; + return Objects.equal(this.partitionMap, other.partitionMap); + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/IBatchCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/trident/IBatchCoordinator.java new file mode 100644 index 000000000..04231f430 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/IBatchCoordinator.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import java.io.Serializable; + +public interface IBatchCoordinator extends Serializable { + boolean isReady(long txid); + + void close(); +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java b/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java new file mode 100644 index 000000000..3e018d995 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +public interface IBrokerReader { + + GlobalPartitionInformation getCurrentBrokers(); + + void close(); +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/MaxMetric.java b/external/storm-kafka/src/jvm/storm/kafka/trident/MaxMetric.java new file mode 100644 index 000000000..60d7c7b52 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/MaxMetric.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + + +import backtype.storm.metric.api.ICombiner; + +public class MaxMetric implements ICombiner { + @Override + public Long identity() { + return null; + } + + @Override + public Long combine(Long l1, Long l2) { + if (l1 == null) { + return l2; + } + if (l2 == null) { + return l1; + } + return Math.max(l1, l2); + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java b/external/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java new file mode 100644 index 000000000..136e7d252 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import storm.kafka.Partition; +import storm.trident.spout.IOpaquePartitionedTridentSpout; + +import java.util.Map; +import java.util.UUID; + + +public class OpaqueTridentKafkaSpout implements IOpaquePartitionedTridentSpout { + + + TridentKafkaConfig _config; + String _topologyInstanceId = UUID.randomUUID().toString(); + + public OpaqueTridentKafkaSpout(TridentKafkaConfig config) { + _config = config; + } + + @Override + public IOpaquePartitionedTridentSpout.Emitter getEmitter(Map conf, TopologyContext context) { + return new TridentKafkaEmitter(conf, context, _config, _topologyInstanceId).asOpaqueEmitter(); + } + + @Override + public IOpaquePartitionedTridentSpout.Coordinator getCoordinator(Map conf, TopologyContext tc) { + return new storm.kafka.trident.Coordinator(conf, _config); + } + + @Override + public Fields getOutputFields() { + return _config.scheme.getOutputFields(); + } + + @Override + public Map getComponentConfiguration() { + return null; + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/StaticBrokerReader.java b/external/storm-kafka/src/jvm/storm/kafka/trident/StaticBrokerReader.java new file mode 100644 index 000000000..d1673f107 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/StaticBrokerReader.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +public class StaticBrokerReader implements IBrokerReader { + + private GlobalPartitionInformation brokers = new GlobalPartitionInformation(); + + public StaticBrokerReader(GlobalPartitionInformation partitionInformation) { + this.brokers = partitionInformation; + } + + @Override + public GlobalPartitionInformation getCurrentBrokers() { + return brokers; + } + + @Override + public void close() { + } +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java b/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java new file mode 100644 index 000000000..7a44bdd72 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import storm.kafka.Partition; +import storm.trident.spout.IPartitionedTridentSpout; + +import java.util.Map; +import java.util.UUID; + + +public class TransactionalTridentKafkaSpout implements IPartitionedTridentSpout { + + TridentKafkaConfig _config; + String _topologyInstanceId = UUID.randomUUID().toString(); + + public TransactionalTridentKafkaSpout(TridentKafkaConfig config) { + _config = config; + } + + + @Override + public IPartitionedTridentSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { + return new storm.kafka.trident.Coordinator(conf, _config); + } + + @Override + public IPartitionedTridentSpout.Emitter getEmitter(Map conf, TopologyContext context) { + return new TridentKafkaEmitter(conf, context, _config, _topologyInstanceId).asTransactionalEmitter(); + } + + @Override + public Fields getOutputFields() { + return _config.scheme.getOutputFields(); + } + + @Override + public Map getComponentConfiguration() { + return null; + } +} \ No newline at end of file diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaConfig.java b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaConfig.java new file mode 100644 index 000000000..3878cc890 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaConfig.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import storm.kafka.BrokerHosts; +import storm.kafka.KafkaConfig; + + +public class TridentKafkaConfig extends KafkaConfig { + + + public final IBatchCoordinator coordinator = new DefaultCoordinator(); + + public TridentKafkaConfig(BrokerHosts hosts, String topic) { + super(hosts, topic); + } + + public TridentKafkaConfig(BrokerHosts hosts, String topic, String clientId) { + super(hosts, topic, clientId); + } + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaEmitter.java b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaEmitter.java new file mode 100644 index 000000000..8c5768148 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaEmitter.java @@ -0,0 +1,269 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import backtype.storm.Config; +import backtype.storm.metric.api.CombinedMetric; +import backtype.storm.metric.api.MeanReducer; +import backtype.storm.metric.api.ReducedMetric; +import backtype.storm.task.TopologyContext; +import backtype.storm.utils.Utils; +import com.google.common.collect.ImmutableMap; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.message.Message; +import kafka.message.MessageAndOffset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.DynamicPartitionConnections; +import storm.kafka.FailedFetchException; +import storm.kafka.KafkaUtils; +import storm.kafka.Partition; +import storm.trident.operation.TridentCollector; +import storm.trident.spout.IOpaquePartitionedTridentSpout; +import storm.trident.spout.IPartitionedTridentSpout; +import storm.trident.topology.TransactionAttempt; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +public class TridentKafkaEmitter { + + public static final Logger LOG = LoggerFactory.getLogger(TridentKafkaEmitter.class); + + private DynamicPartitionConnections _connections; + private String _topologyName; + private KafkaUtils.KafkaOffsetMetric _kafkaOffsetMetric; + private ReducedMetric _kafkaMeanFetchLatencyMetric; + private CombinedMetric _kafkaMaxFetchLatencyMetric; + private TridentKafkaConfig _config; + private String _topologyInstanceId; + + public TridentKafkaEmitter(Map conf, TopologyContext context, TridentKafkaConfig config, String topologyInstanceId) { + _config = config; + _topologyInstanceId = topologyInstanceId; + _connections = new DynamicPartitionConnections(_config, KafkaUtils.makeBrokerReader(conf, _config)); + _topologyName = (String) conf.get(Config.TOPOLOGY_NAME); + _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_config.topic, _connections); + context.registerMetric("kafkaOffset", _kafkaOffsetMetric, _config.metricsTimeBucketSizeInSecs); + _kafkaMeanFetchLatencyMetric = context.registerMetric("kafkaFetchAvg", new MeanReducer(), _config.metricsTimeBucketSizeInSecs); + _kafkaMaxFetchLatencyMetric = context.registerMetric("kafkaFetchMax", new MaxMetric(), _config.metricsTimeBucketSizeInSecs); + } + + + private Map failFastEmitNewPartitionBatch(TransactionAttempt attempt, TridentCollector collector, Partition partition, Map lastMeta) { + SimpleConsumer consumer = _connections.register(partition); + Map ret = doEmitNewPartitionBatch(consumer, partition, collector, lastMeta); + _kafkaOffsetMetric.setLatestEmittedOffset(partition, (Long) ret.get("offset")); + return ret; + } + + private Map emitNewPartitionBatch(TransactionAttempt attempt, TridentCollector collector, Partition partition, Map lastMeta) { + try { + return failFastEmitNewPartitionBatch(attempt, collector, partition, lastMeta); + } catch (FailedFetchException e) { + LOG.warn("Failed to fetch from partition " + partition); + if (lastMeta == null) { + return null; + } else { + Map ret = new HashMap(); + ret.put("offset", lastMeta.get("nextOffset")); + ret.put("nextOffset", lastMeta.get("nextOffset")); + ret.put("partition", partition.partition); + ret.put("broker", ImmutableMap.of("host", partition.host.host, "port", partition.host.port)); + ret.put("topic", _config.topic); + ret.put("topology", ImmutableMap.of("name", _topologyName, "id", _topologyInstanceId)); + return ret; + } + } + } + + private Map doEmitNewPartitionBatch(SimpleConsumer consumer, Partition partition, TridentCollector collector, Map lastMeta) { + long offset; + if (lastMeta != null) { + String lastInstanceId = null; + Map lastTopoMeta = (Map) lastMeta.get("topology"); + if (lastTopoMeta != null) { + lastInstanceId = (String) lastTopoMeta.get("id"); + } + if (_config.forceFromStart && !_topologyInstanceId.equals(lastInstanceId)) { + offset = KafkaUtils.getOffset(consumer, _config.topic, partition.partition, _config.startOffsetTime); + } else { + offset = (Long) lastMeta.get("nextOffset"); + } + } else { + offset = KafkaUtils.getOffset(consumer, _config.topic, partition.partition, _config); + } + ByteBufferMessageSet msgs = fetchMessages(consumer, partition, offset); + long endoffset = offset; + for (MessageAndOffset msg : msgs) { + emit(collector, msg.message()); + endoffset = msg.nextOffset(); + } + Map newMeta = new HashMap(); + newMeta.put("offset", offset); + newMeta.put("nextOffset", endoffset); + newMeta.put("instanceId", _topologyInstanceId); + newMeta.put("partition", partition.partition); + newMeta.put("broker", ImmutableMap.of("host", partition.host.host, "port", partition.host.port)); + newMeta.put("topic", _config.topic); + newMeta.put("topology", ImmutableMap.of("name", _topologyName, "id", _topologyInstanceId)); + return newMeta; + } + + private ByteBufferMessageSet fetchMessages(SimpleConsumer consumer, Partition partition, long offset) { + long start = System.nanoTime(); + ByteBufferMessageSet msgs = KafkaUtils.fetchMessages(_config, consumer, partition, offset); + long end = System.nanoTime(); + long millis = (end - start) / 1000000; + _kafkaMeanFetchLatencyMetric.update(millis); + _kafkaMaxFetchLatencyMetric.update(millis); + return msgs; + } + + /** + * re-emit the batch described by the meta data provided + * + * @param attempt + * @param collector + * @param partition + * @param meta + */ + private void reEmitPartitionBatch(TransactionAttempt attempt, TridentCollector collector, Partition partition, Map meta) { + LOG.info("re-emitting batch, attempt " + attempt); + String instanceId = (String) meta.get("instanceId"); + if (!_config.forceFromStart || instanceId.equals(_topologyInstanceId)) { + SimpleConsumer consumer = _connections.register(partition); + long offset = (Long) meta.get("offset"); + long nextOffset = (Long) meta.get("nextOffset"); + ByteBufferMessageSet msgs = fetchMessages(consumer, partition, offset); + for (MessageAndOffset msg : msgs) { + if (offset == nextOffset) { + break; + } + if (offset > nextOffset) { + throw new RuntimeException("Error when re-emitting batch. overshot the end offset"); + } + emit(collector, msg.message()); + offset = msg.nextOffset(); + } + } + } + + private void emit(TridentCollector collector, Message msg) { + Iterable> values = KafkaUtils.generateTuples(_config, msg); + if (values != null) { + for (List value : values) { + collector.emit(value); + } + } + } + + private void clear() { + _connections.clear(); + } + + private List orderPartitions(GlobalPartitionInformation partitions) { + return partitions.getOrderedPartitions(); + } + + private void refresh(List list) { + _connections.clear(); + _kafkaOffsetMetric.refreshPartitions(new HashSet(list)); + } + + + public IOpaquePartitionedTridentSpout.Emitter asOpaqueEmitter() { + + return new IOpaquePartitionedTridentSpout.Emitter() { + + /** + * Emit a batch of tuples for a partition/transaction. + * + * Return the metadata describing this batch that will be used as lastPartitionMeta + * for defining the parameters of the next batch. + */ + @Override + public Map emitPartitionBatch(TransactionAttempt transactionAttempt, TridentCollector tridentCollector, Partition partition, Map map) { + return emitNewPartitionBatch(transactionAttempt, tridentCollector, partition, map); + } + + @Override + public void refreshPartitions(List partitions) { + refresh(partitions); + } + + @Override + public List getOrderedPartitions(GlobalPartitionInformation partitionInformation) { + return orderPartitions(partitionInformation); + } + + @Override + public void close() { + clear(); + } + }; + } + + public IPartitionedTridentSpout.Emitter asTransactionalEmitter() { + return new IPartitionedTridentSpout.Emitter() { + + /** + * Emit a batch of tuples for a partition/transaction that's never been emitted before. + * Return the metadata that can be used to reconstruct this partition/batch in the future. + */ + @Override + public Map emitPartitionBatchNew(TransactionAttempt transactionAttempt, TridentCollector tridentCollector, Partition partition, Map map) { + return failFastEmitNewPartitionBatch(transactionAttempt, tridentCollector, partition, map); + } + + /** + * Emit a batch of tuples for a partition/transaction that has been emitted before, using + * the metadata created when it was first emitted. + */ + @Override + public void emitPartitionBatch(TransactionAttempt transactionAttempt, TridentCollector tridentCollector, Partition partition, Map map) { + reEmitPartitionBatch(transactionAttempt, tridentCollector, partition, map); + } + + /** + * This method is called when this task is responsible for a new set of partitions. Should be used + * to manage things like connections to brokers. + */ + @Override + public void refreshPartitions(List partitions) { + refresh(partitions); + } + + @Override + public List getOrderedPartitions(GlobalPartitionInformation partitionInformation) { + return orderPartitions(partitionInformation); + } + + @Override + public void close() { + clear(); + } + }; + + } + + +} diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/ZkBrokerReader.java b/external/storm-kafka/src/jvm/storm/kafka/trident/ZkBrokerReader.java new file mode 100644 index 000000000..db1132893 --- /dev/null +++ b/external/storm-kafka/src/jvm/storm/kafka/trident/ZkBrokerReader.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.kafka.trident; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.kafka.DynamicBrokersReader; +import storm.kafka.ZkHosts; + +import java.util.Map; + + +public class ZkBrokerReader implements IBrokerReader { + + public static final Logger LOG = LoggerFactory.getLogger(ZkBrokerReader.class); + + GlobalPartitionInformation cachedBrokers; + DynamicBrokersReader reader; + long lastRefreshTimeMs; + + + long refreshMillis; + + public ZkBrokerReader(Map conf, String topic, ZkHosts hosts) { + reader = new DynamicBrokersReader(conf, hosts.brokerZkStr, hosts.brokerZkPath, topic); + cachedBrokers = reader.getBrokerInfo(); + lastRefreshTimeMs = System.currentTimeMillis(); + refreshMillis = hosts.refreshFreqSecs * 1000L; + + } + + @Override + public GlobalPartitionInformation getCurrentBrokers() { + long currTime = System.currentTimeMillis(); + if (currTime > lastRefreshTimeMs + refreshMillis) { + LOG.info("brokers need refreshing because " + refreshMillis + "ms have expired"); + cachedBrokers = reader.getBrokerInfo(); + lastRefreshTimeMs = currTime; + } + return cachedBrokers; + } + + @Override + public void close() { + reader.close(); + } +} diff --git a/external/storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java b/external/storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java new file mode 100644 index 000000000..d03bab30d --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java @@ -0,0 +1,155 @@ +package storm.kafka; + +import backtype.storm.Config; +import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.CuratorFrameworkFactory; +import com.netflix.curator.retry.ExponentialBackoffRetry; +import com.netflix.curator.test.TestingServer; +import com.netflix.curator.utils.ZKPaths; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import storm.kafka.trident.GlobalPartitionInformation; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Date: 16/05/2013 + * Time: 20:35 + */ +public class DynamicBrokersReaderTest { + private DynamicBrokersReader dynamicBrokersReader; + private String masterPath = "/brokers"; + private String topic = "testing"; + private CuratorFramework zookeeper; + private TestingServer server; + + @Before + public void setUp() throws Exception { + server = new TestingServer(); + String connectionString = server.getConnectString(); + Map conf = new HashMap(); + conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 1000); + conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4); + conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 5); + ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(1000, 3); + zookeeper = CuratorFrameworkFactory.newClient(connectionString, retryPolicy); + dynamicBrokersReader = new DynamicBrokersReader(conf, connectionString, masterPath, topic); + zookeeper.start(); + } + + @After + public void tearDown() throws Exception { + dynamicBrokersReader.close(); + zookeeper.close(); + server.close(); + } + + private void addPartition(int id, String host, int port) throws Exception { + writePartitionId(id); + writeLeader(id, 0); + writeLeaderDetails(0, host, port); + } + + private void addPartition(int id, int leader, String host, int port) throws Exception { + writePartitionId(id); + writeLeader(id, leader); + writeLeaderDetails(leader, host, port); + } + + private void writePartitionId(int id) throws Exception { + String path = dynamicBrokersReader.partitionPath(); + writeDataToPath(path, ("" + id)); + } + + private void writeDataToPath(String path, String data) throws Exception { + ZKPaths.mkdirs(zookeeper.getZookeeperClient().getZooKeeper(), path); + zookeeper.setData().forPath(path, data.getBytes()); + } + + private void writeLeader(int id, int leaderId) throws Exception { + String path = dynamicBrokersReader.partitionPath() + "/" + id + "/state"; + String value = " { \"controller_epoch\":4, \"isr\":[ 1, 0 ], \"leader\":" + leaderId + ", \"leader_epoch\":1, \"version\":1 }"; + writeDataToPath(path, value); + } + + private void writeLeaderDetails(int leaderId, String host, int port) throws Exception { + String path = dynamicBrokersReader.brokerPath() + "/" + leaderId; + String value = "{ \"host\":\"" + host + "\", \"jmx_port\":9999, \"port\":" + port + ", \"version\":1 }"; + writeDataToPath(path, value); + } + + @Test + public void testGetBrokerInfo() throws Exception { + String host = "localhost"; + int port = 9092; + int partition = 0; + addPartition(partition, host, port); + GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo(); + assertEquals(1, brokerInfo.getOrderedPartitions().size()); + assertEquals(port, brokerInfo.getBrokerFor(partition).port); + assertEquals(host, brokerInfo.getBrokerFor(partition).host); + } + + + @Test + public void testMultiplePartitionsOnDifferentHosts() throws Exception { + String host = "localhost"; + int port = 9092; + int secondPort = 9093; + int partition = 0; + int secondPartition = partition + 1; + addPartition(partition, 0, host, port); + addPartition(secondPartition, 1, host, secondPort); + + GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo(); + assertEquals(2, brokerInfo.getOrderedPartitions().size()); + + assertEquals(port, brokerInfo.getBrokerFor(partition).port); + assertEquals(host, brokerInfo.getBrokerFor(partition).host); + + assertEquals(secondPort, brokerInfo.getBrokerFor(secondPartition).port); + assertEquals(host, brokerInfo.getBrokerFor(secondPartition).host); + } + + + @Test + public void testMultiplePartitionsOnSameHost() throws Exception { + String host = "localhost"; + int port = 9092; + int partition = 0; + int secondPartition = partition + 1; + addPartition(partition, 0, host, port); + addPartition(secondPartition, 0, host, port); + + GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo(); + assertEquals(2, brokerInfo.getOrderedPartitions().size()); + + assertEquals(port, brokerInfo.getBrokerFor(partition).port); + assertEquals(host, brokerInfo.getBrokerFor(partition).host); + + assertEquals(port, brokerInfo.getBrokerFor(secondPartition).port); + assertEquals(host, brokerInfo.getBrokerFor(secondPartition).host); + } + + @Test + public void testSwitchHostForPartition() throws Exception { + String host = "localhost"; + int port = 9092; + int partition = 0; + addPartition(partition, host, port); + GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo(); + assertEquals(port, brokerInfo.getBrokerFor(partition).port); + assertEquals(host, brokerInfo.getBrokerFor(partition).host); + + String newHost = host + "switch"; + int newPort = port + 1; + addPartition(partition, newHost, newPort); + brokerInfo = dynamicBrokersReader.getBrokerInfo(); + assertEquals(newPort, brokerInfo.getBrokerFor(partition).port); + assertEquals(newHost, brokerInfo.getBrokerFor(partition).host); + } +} diff --git a/external/storm-kafka/src/test/storm/kafka/KafkaErrorTest.java b/external/storm-kafka/src/test/storm/kafka/KafkaErrorTest.java new file mode 100644 index 000000000..e5f9db280 --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/KafkaErrorTest.java @@ -0,0 +1,39 @@ +package storm.kafka; + +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +/** + * Date: 12/01/2014 + * Time: 18:09 + */ +public class KafkaErrorTest { + + @Test + public void getError() { + assertThat(KafkaError.getError(0), is(equalTo(KafkaError.NO_ERROR))); + } + + @Test + public void offsetMetaDataTooLarge() { + assertThat(KafkaError.getError(12), is(equalTo(KafkaError.OFFSET_METADATA_TOO_LARGE))); + } + + @Test + public void unknownNegative() { + assertThat(KafkaError.getError(-1), is(equalTo(KafkaError.UNKNOWN))); + } + + @Test + public void unknownPositive() { + assertThat(KafkaError.getError(75), is(equalTo(KafkaError.UNKNOWN))); + } + + @Test + public void unknown() { + assertThat(KafkaError.getError(13), is(equalTo(KafkaError.UNKNOWN))); + } +} diff --git a/external/storm-kafka/src/test/storm/kafka/KafkaTestBroker.java b/external/storm-kafka/src/test/storm/kafka/KafkaTestBroker.java new file mode 100644 index 000000000..db4663d27 --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/KafkaTestBroker.java @@ -0,0 +1,58 @@ +package storm.kafka; + +import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.CuratorFrameworkFactory; +import com.netflix.curator.retry.ExponentialBackoffRetry; +import com.netflix.curator.test.TestingServer; +import kafka.server.KafkaServerStartable; + +import java.io.IOException; +import java.util.Properties; + +/** + * Date: 11/01/2014 + * Time: 13:15 + */ +public class KafkaTestBroker { + + private final int port = 49123; + private KafkaServerStartable kafka; + private TestingServer server; + private String zookeeperConnectionString; + + public KafkaTestBroker() { + try { + server = new TestingServer(); + zookeeperConnectionString = server.getConnectString(); + ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(1000, 3); + CuratorFramework zookeeper = CuratorFrameworkFactory.newClient(zookeeperConnectionString, retryPolicy); + zookeeper.start(); + Properties p = new Properties(); + p.setProperty("zookeeper.connect", zookeeperConnectionString); + p.setProperty("broker.id", "0"); + p.setProperty("port", "" + port); + kafka.server.KafkaConfig config = new kafka.server.KafkaConfig(p); + kafka = new KafkaServerStartable(config); + kafka.startup(); + } catch (Exception ex) { + throw new RuntimeException("Could not start test broker", ex); + } + } + + public String getBrokerConnectionString() { + return "localhost:" + port; + } + + public int getPort() { + return port; + } + + public void shutdown() { + kafka.shutdown(); + try { + server.stop(); + } catch (IOException e) { + e.printStackTrace(); + } + } +} diff --git a/external/storm-kafka/src/test/storm/kafka/KafkaUtilsTest.java b/external/storm-kafka/src/test/storm/kafka/KafkaUtilsTest.java new file mode 100644 index 000000000..25fa618fd --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/KafkaUtilsTest.java @@ -0,0 +1,221 @@ +package storm.kafka; + +import backtype.storm.spout.SchemeAsMultiScheme; +import backtype.storm.utils.Utils; +import com.google.common.collect.ImmutableMap; +import kafka.api.OffsetRequest; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.javaapi.producer.Producer; +import kafka.message.MessageAndOffset; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import storm.kafka.trident.GlobalPartitionInformation; + +import java.util.List; +import java.util.Properties; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +public class KafkaUtilsTest { + + private KafkaTestBroker broker; + private SimpleConsumer simpleConsumer; + private KafkaConfig config; + private BrokerHosts brokerHosts; + + @Before + public void setup() { + broker = new KafkaTestBroker(); + GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(); + globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString())); + brokerHosts = new StaticHosts(globalPartitionInformation); + config = new KafkaConfig(brokerHosts, "testTopic"); + simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient"); + } + + @After + public void shutdown() { + simpleConsumer.close(); + broker.shutdown(); + } + + + @Test(expected = FailedFetchException.class) + public void topicDoesNotExist() throws Exception { + KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), 0); + } + + @Test(expected = FailedFetchException.class) + public void brokerIsDown() throws Exception { + int port = broker.getPort(); + broker.shutdown(); + SimpleConsumer simpleConsumer = new SimpleConsumer("localhost", port, 100, 1024, "testClient"); + KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), OffsetRequest.LatestTime()); + } + + @Test + public void fetchMessage() throws Exception { + String value = "test"; + createTopicAndSendMessage(value); + long offset = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.LatestTime()) - 1; + ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(config, simpleConsumer, + new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), offset); + String message = new String(Utils.toByteArray(messageAndOffsets.iterator().next().message().payload())); + assertThat(message, is(equalTo(value))); + } + + @Test(expected = FailedFetchException.class) + public void fetchMessagesWithInvalidOffsetAndDefaultHandlingDisabled() throws Exception { + config.useStartOffsetTimeIfOffsetOutOfRange = false; + KafkaUtils.fetchMessages(config, simpleConsumer, + new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), -99); + } + + @Test + public void fetchMessagesWithInvalidOffsetAndDefaultHandlingEnabled() throws Exception { + config = new KafkaConfig(brokerHosts, "newTopic"); + String value = "test"; + createTopicAndSendMessage(value); + ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(config, simpleConsumer, + new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), -99); + String message = new String(Utils.toByteArray(messageAndOffsets.iterator().next().message().payload())); + assertThat(message, is(equalTo(value))); + } + + @Test + public void getOffsetFromConfigAndDontForceFromStart() { + config.forceFromStart = false; + config.startOffsetTime = OffsetRequest.EarliestTime(); + createTopicAndSendMessage(); + long latestOffset = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.LatestTime()); + long offsetFromConfig = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, config); + assertThat(latestOffset, is(equalTo(offsetFromConfig))); + } + + @Test + public void getOffsetFromConfigAndFroceFromStart() { + config.forceFromStart = true; + config.startOffsetTime = OffsetRequest.EarliestTime(); + createTopicAndSendMessage(); + long earliestOffset = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.EarliestTime()); + long offsetFromConfig = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, config); + assertThat(earliestOffset, is(equalTo(offsetFromConfig))); + } + + @Test + public void generateTuplesWithoutKeyAndKeyValueScheme() { + config.scheme = new KeyValueSchemeAsMultiScheme(new StringKeyValueScheme()); + runGetValueOnlyTuplesTest(); + } + + @Test + public void generateTuplesWithKeyAndKeyValueScheme() { + config.scheme = new KeyValueSchemeAsMultiScheme(new StringKeyValueScheme()); + String value = "value"; + String key = "key"; + createTopicAndSendMessage(key, value); + ByteBufferMessageSet messageAndOffsets = getLastMessage(); + for (MessageAndOffset msg : messageAndOffsets) { + Iterable> lists = KafkaUtils.generateTuples(config, msg.message()); + assertEquals(ImmutableMap.of(key, value), lists.iterator().next().get(0)); + } + } + + @Test + public void generateTupelsWithValueScheme() { + config.scheme = new SchemeAsMultiScheme(new StringScheme()); + runGetValueOnlyTuplesTest(); + } + + @Test + public void generateTuplesWithValueSchemeAndKeyValueMessage() { + config.scheme = new SchemeAsMultiScheme(new StringScheme()); + String value = "value"; + String key = "key"; + createTopicAndSendMessage(key, value); + ByteBufferMessageSet messageAndOffsets = getLastMessage(); + for (MessageAndOffset msg : messageAndOffsets) { + Iterable> lists = KafkaUtils.generateTuples(config, msg.message()); + assertEquals(value, lists.iterator().next().get(0)); + } + } + + private ByteBufferMessageSet getLastMessage() { + long offsetOfLastMessage = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.LatestTime()) - 1; + return KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), offsetOfLastMessage); + } + + private void runGetValueOnlyTuplesTest() { + String value = "value"; + createTopicAndSendMessage(null, value); + ByteBufferMessageSet messageAndOffsets = getLastMessage(); + for (MessageAndOffset msg : messageAndOffsets) { + Iterable> lists = KafkaUtils.generateTuples(config, msg.message()); + assertEquals(value, lists.iterator().next().get(0)); + } + } + + + private void createTopicAndSendMessage() { + createTopicAndSendMessage(null, "someValue"); + } + + private void createTopicAndSendMessage(String value) { + createTopicAndSendMessage(null, value); + } + + private void createTopicAndSendMessage(String key, String value) { + Properties p = new Properties(); + p.setProperty("metadata.broker.list", "localhost:49123"); + p.setProperty("serializer.class", "kafka.serializer.StringEncoder"); + ProducerConfig producerConfig = new ProducerConfig(p); + Producer producer = new Producer(producerConfig); + producer.send(new KeyedMessage(config.topic, key, value)); + } + + + @Test + public void assignOnePartitionPerTask() { + runPartitionToTaskMappingTest(16, 1); + } + + @Test + public void assignTwoPartitionsPerTask() { + runPartitionToTaskMappingTest(16, 2); + } + + @Test + public void assignAllPartitionsToOneTask() { + runPartitionToTaskMappingTest(32, 32); + } + + + public void runPartitionToTaskMappingTest(int numPartitions, int partitionsPerTask) { + GlobalPartitionInformation globalPartitionInformation = TestUtils.buildPartitionInfo(numPartitions); + int numTasks = numPartitions / partitionsPerTask; + for (int i = 0 ; i < numTasks ; i++) { + assertEquals(partitionsPerTask, KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, numTasks, i).size()); + } + } + + @Test + public void moreTasksThanPartitions() { + GlobalPartitionInformation globalPartitionInformation = TestUtils.buildPartitionInfo(1); + int numTasks = 2; + assertEquals(1, KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, numTasks, 0).size()); + assertEquals(0, KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, numTasks, 1).size()); + } + + @Test (expected = IllegalArgumentException.class ) + public void assignInvalidTask() { + GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(); + KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, 1, 1); + } +} diff --git a/external/storm-kafka/src/test/storm/kafka/StringKeyValueSchemeTest.java b/external/storm-kafka/src/test/storm/kafka/StringKeyValueSchemeTest.java new file mode 100644 index 000000000..4413c7bd7 --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/StringKeyValueSchemeTest.java @@ -0,0 +1,38 @@ +package storm.kafka; + +import backtype.storm.tuple.Fields; +import com.google.common.collect.ImmutableMap; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StringKeyValueSchemeTest { + + private StringKeyValueScheme scheme = new StringKeyValueScheme(); + + @Test + public void testDeserialize() throws Exception { + assertEquals(Arrays.asList("test"), scheme.deserialize("test".getBytes())); + } + + @Test + public void testGetOutputFields() throws Exception { + Fields outputFields = scheme.getOutputFields(); + assertTrue(outputFields.contains(StringScheme.STRING_SCHEME_KEY)); + assertEquals(1, outputFields.size()); + } + + @Test + public void testDeserializeWithNullKeyAndValue() throws Exception { + assertEquals(Arrays.asList("test"), scheme.deserializeKeyAndValue(null, "test".getBytes())); + } + + @Test + public void testDeserializeWithKeyAndValue() throws Exception { + assertEquals(Arrays.asList(ImmutableMap.of("key", "test")), + scheme.deserializeKeyAndValue("key".getBytes(), "test".getBytes())); + } +} diff --git a/external/storm-kafka/src/test/storm/kafka/TestUtils.java b/external/storm-kafka/src/test/storm/kafka/TestUtils.java new file mode 100644 index 000000000..860d96d60 --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/TestUtils.java @@ -0,0 +1,20 @@ +package storm.kafka; + +import storm.kafka.trident.GlobalPartitionInformation; + +public class TestUtils { + + public static GlobalPartitionInformation buildPartitionInfo(int numPartitions) { + return buildPartitionInfo(numPartitions, 9092); + } + + + public static GlobalPartitionInformation buildPartitionInfo(int numPartitions, int brokerPort) { + GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(); + for (int i = 0; i < numPartitions; i++) { + globalPartitionInformation.addPartition(i, Broker.fromString("broker-" + i + " :" + brokerPort)); + } + return globalPartitionInformation; + } + +} diff --git a/external/storm-kafka/src/test/storm/kafka/ZkCoordinatorTest.java b/external/storm-kafka/src/test/storm/kafka/ZkCoordinatorTest.java new file mode 100644 index 000000000..c08ce8253 --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/ZkCoordinatorTest.java @@ -0,0 +1,130 @@ +package storm.kafka; + +import backtype.storm.Config; +import com.netflix.curator.test.TestingServer; +import kafka.javaapi.consumer.SimpleConsumer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.*; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.when; + +public class ZkCoordinatorTest { + + + @Mock + private DynamicBrokersReader reader; + + @Mock + private DynamicPartitionConnections dynamicPartitionConnections; + + private KafkaTestBroker broker = new KafkaTestBroker(); + private TestingServer server; + private Map stormConf = new HashMap(); + private SpoutConfig spoutConfig; + private ZkState state; + private SimpleConsumer simpleConsumer; + + @Before + public void setUp() throws Exception { + MockitoAnnotations.initMocks(this); + server = new TestingServer(); + String connectionString = server.getConnectString(); + ZkHosts hosts = new ZkHosts(connectionString); + hosts.refreshFreqSecs = 1; + spoutConfig = new SpoutConfig(hosts, "topic", "/test", "id"); + Map conf = buildZookeeperConfig(server); + state = new ZkState(conf); + simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient"); + when(dynamicPartitionConnections.register(any(Broker.class), anyInt())).thenReturn(simpleConsumer); + } + + private Map buildZookeeperConfig(TestingServer server) { + Map conf = new HashMap(); + conf.put(Config.TRANSACTIONAL_ZOOKEEPER_PORT, server.getPort()); + conf.put(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Arrays.asList("localhost")); + conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 20000); + conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 3); + conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 30); + return conf; + } + + @After + public void shutdown() throws Exception { + simpleConsumer.close(); + broker.shutdown(); + server.stop(); + } + + @Test + public void testOnePartitionPerTask() throws Exception { + int totalTasks = 64; + int partitionsPerTask = 1; + List coordinatorList = buildCoordinators(totalTasks / partitionsPerTask); + when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfo(totalTasks)); + for (ZkCoordinator coordinator : coordinatorList) { + List myManagedPartitions = coordinator.getMyManagedPartitions(); + assertEquals(partitionsPerTask, myManagedPartitions.size()); + assertEquals(coordinator._taskIndex, myManagedPartitions.get(0).getPartition().partition); + } + } + + + @Test + public void testPartitionsChange() throws Exception { + final int totalTasks = 64; + int partitionsPerTask = 2; + List coordinatorList = buildCoordinators(totalTasks / partitionsPerTask); + when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfo(totalTasks, 9092)); + List> partitionManagersBeforeRefresh = getPartitionManagers(coordinatorList); + waitForRefresh(); + when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfo(totalTasks, 9093)); + List> partitionManagersAfterRefresh = getPartitionManagers(coordinatorList); + assertEquals(partitionManagersAfterRefresh.size(), partitionManagersAfterRefresh.size()); + Iterator> iterator = partitionManagersAfterRefresh.iterator(); + for (List partitionManagersBefore : partitionManagersBeforeRefresh) { + List partitionManagersAfter = iterator.next(); + assertPartitionsAreDifferent(partitionManagersBefore, partitionManagersAfter, partitionsPerTask); + } + } + + private void assertPartitionsAreDifferent(List partitionManagersBefore, List partitionManagersAfter, int partitionsPerTask) { + assertEquals(partitionsPerTask, partitionManagersBefore.size()); + assertEquals(partitionManagersBefore.size(), partitionManagersAfter.size()); + for (int i = 0; i < partitionsPerTask; i++) { + assertNotEquals(partitionManagersBefore.get(i).getPartition(), partitionManagersAfter.get(i).getPartition()); + } + + } + + private List> getPartitionManagers(List coordinatorList) { + List> partitions = new ArrayList(); + for (ZkCoordinator coordinator : coordinatorList) { + partitions.add(coordinator.getMyManagedPartitions()); + } + return partitions; + } + + private void waitForRefresh() throws InterruptedException { + Thread.sleep(((ZkHosts) spoutConfig.hosts).refreshFreqSecs * 1000 + 1); + } + + private List buildCoordinators(int totalTasks) { + List coordinatorList = new ArrayList(); + for (int i = 0; i < totalTasks; i++) { + ZkCoordinator coordinator = new ZkCoordinator(dynamicPartitionConnections, stormConf, spoutConfig, state, i, totalTasks, "test-id", reader); + coordinatorList.add(coordinator); + } + return coordinatorList; + } + + +} diff --git a/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java new file mode 100644 index 000000000..fa5a104a7 --- /dev/null +++ b/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java @@ -0,0 +1,171 @@ +package storm.kafka.bolt; + +import backtype.storm.Config; +import backtype.storm.task.GeneralTopologyContext; +import backtype.storm.task.IOutputCollector; +import backtype.storm.task.OutputCollector; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.TupleImpl; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; +import kafka.api.OffsetRequest; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.message.Message; +import kafka.message.MessageAndOffset; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import storm.kafka.*; +import storm.kafka.trident.GlobalPartitionInformation; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.verify; + +public class KafkaBoltTest { + + private static final String TEST_TOPIC = "test-topic"; + private KafkaTestBroker broker; + private KafkaBolt bolt; + private Config config = new Config(); + private KafkaConfig kafkaConfig; + private SimpleConsumer simpleConsumer; + + @Mock + private IOutputCollector collector; + + @Before + public void initMocks() { + MockitoAnnotations.initMocks(this); + broker = new KafkaTestBroker(); + setupKafkaConsumer(); + config.put(KafkaBolt.TOPIC, TEST_TOPIC); + bolt = generateStringSerializerBolt(); + } + + @After + public void shutdown() { + simpleConsumer.close(); + broker.shutdown(); + } + + + private void setupKafkaConsumer() { + GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(); + globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString())); + BrokerHosts brokerHosts = new StaticHosts(globalPartitionInformation); + kafkaConfig = new KafkaConfig(brokerHosts, TEST_TOPIC); + simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient"); + } + + @Test + public void executeWithKey() throws Exception { + String message = "value-123"; + String key = "key-123"; + Tuple tuple = generateTestTuple(key, message); + bolt.execute(tuple); + verify(collector).ack(tuple); + verifyMessage(key, message); + } + + @Test + public void executeWithByteArrayKeyAndMessage() { + bolt = generateDefaultSerializerBolt(); + String keyString = "test-key"; + String messageString = "test-message"; + byte[] key = keyString.getBytes(); + byte[] message = messageString.getBytes(); + Tuple tuple = generateTestTuple(key, message); + bolt.execute(tuple); + verify(collector).ack(tuple); + verifyMessage(keyString, messageString); + } + + private KafkaBolt generateStringSerializerBolt() { + KafkaBolt bolt = new KafkaBolt(); + Properties props = new Properties(); + props.put("metadata.broker.list", broker.getBrokerConnectionString()); + props.put("request.required.acks", "1"); + props.put("serializer.class", "kafka.serializer.StringEncoder"); + config.put(KafkaBolt.KAFKA_BROKER_PROPERTIES, props); + bolt.prepare(config, null, new OutputCollector(collector)); + return bolt; + } + + private KafkaBolt generateDefaultSerializerBolt() { + KafkaBolt bolt = new KafkaBolt(); + Properties props = new Properties(); + props.put("metadata.broker.list", broker.getBrokerConnectionString()); + props.put("request.required.acks", "1"); + config.put(KafkaBolt.KAFKA_BROKER_PROPERTIES, props); + bolt.prepare(config, null, new OutputCollector(collector)); + return bolt; + } + + @Test + public void executeWithoutKey() throws Exception { + String message = "value-234"; + Tuple tuple = generateTestTuple(message); + bolt.execute(tuple); + verify(collector).ack(tuple); + verifyMessage(null, message); + } + + + @Test + public void executeWithBrokerDown() throws Exception { + broker.shutdown(); + String message = "value-234"; + Tuple tuple = generateTestTuple(message); + bolt.execute(tuple); + verify(collector).ack(tuple); + } + + + private boolean verifyMessage(String key, String message) { + long lastMessageOffset = KafkaUtils.getOffset(simpleConsumer, kafkaConfig.topic, 0, OffsetRequest.LatestTime()) - 1; + ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(kafkaConfig, simpleConsumer, + new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), lastMessageOffset); + MessageAndOffset messageAndOffset = messageAndOffsets.iterator().next(); + Message kafkaMessage = messageAndOffset.message(); + ByteBuffer messageKeyBuffer = kafkaMessage.key(); + String keyString = null; + String messageString = new String(Utils.toByteArray(kafkaMessage.payload())); + if (messageKeyBuffer != null) { + keyString = new String(Utils.toByteArray(messageKeyBuffer)); + } + assertEquals(key, keyString); + assertEquals(message, messageString); + return true; + } + + private Tuple generateTestTuple(Object key, Object message) { + TopologyBuilder builder = new TopologyBuilder(); + GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), new HashMap(), new HashMap(), "") { + @Override + public Fields getComponentOutputFields(String componentId, String streamId) { + return new Fields("key", "message"); + } + }; + return new TupleImpl(topologyContext, new Values(key, message), 1, ""); + } + + private Tuple generateTestTuple(Object message) { + TopologyBuilder builder = new TopologyBuilder(); + GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), new HashMap(), new HashMap(), "") { + @Override + public Fields getComponentOutputFields(String componentId, String streamId) { + return new Fields("message"); + } + }; + return new TupleImpl(topologyContext, new Values(message), 1, ""); + } +} diff --git a/pom.xml b/pom.xml index 720fab3b9..b1a3d5ac0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.1-incubating-SNAPSHOT + 0.9.2-incubating-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -114,7 +114,7 @@ jjackson Jason Jackson - jason@cvk.ca + jasonjckn@gmail.com Committer @@ -130,6 +130,8 @@ storm-buildtools/maven-shade-clojure-transformer storm-core + examples/storm-starter + external/storm-kafka @@ -146,21 +148,22 @@ UTF-8 + -Djava.net.preferIPv4Stack=true 1.4.0 1.1.3 0.3.6 - 1.4 + 2.4 2.5 1.1 0.4.1 - 1.0.1 + 1.3.3 1.1 0.3.11 0.2.3 0.0.1 - 1.3.2 + 1.4.0 1.11 4.1.1 0.2.2 @@ -174,7 +177,8 @@ 0.2.3 1.9.5 0.3.0 - + 2.1.3 + 3.4.5 @@ -267,7 +271,7 @@ commons-io commons-io - ${commons-io.verson} + ${commons-io.version} org.apache.commons @@ -284,6 +288,37 @@ clj-time ${clj-time.version} + + org.apache.zookeeper + zookeeper + ${zookeeper.version} + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + javax.jms + jms + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + com.netflix.curator curator-framework @@ -410,13 +445,25 @@ ${mockito.version} test + + org.clojars.runa + conjure + ${conjure.version} + test + org.apache.thrift libthrift 0.7.0 compile - + + + junit + junit + 3.8.1 + test + diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 3b8cd9796..7dfa2a205 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.1-incubating-SNAPSHOT + 0.9.2-incubating-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 20366861e..a63d5b85f 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.1-incubating-SNAPSHOT + 0.9.2-incubating-SNAPSHOT org.apache.storm storm-core @@ -161,6 +161,11 @@ mockito-all test + + org.clojars.runa + conjure + test + src/jvm @@ -190,6 +195,10 @@ test/clj false + true + + none + @@ -205,6 +214,9 @@ test-with-junit + + ${test.extra.args} + @@ -237,7 +249,7 @@ org.apache.thrift:* - odg.apache.storm:* + org.apache.storm:* @@ -250,7 +262,7 @@ - + diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj index 43cb6fe8b..0ffb88820 100644 --- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj @@ -438,6 +438,11 @@ (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) )) +(defn- substitute-worker-childopts [value port] + (let [sub-fn (fn [s] (.replaceAll s "%ID%" (str port)))] + (if (list? value) + (map sub-fn value) + (-> value sub-fn (.split " "))))) (defmethod launch-worker :distributed [supervisor storm-id port worker-id] @@ -447,22 +452,33 @@ stormjar (supervisor-stormjar-path stormroot) storm-conf (read-supervisor-storm-conf conf storm-id) classpath (add-to-classpath (current-classpath) [stormjar]) - childopts (.replaceAll (str (conf WORKER-CHILDOPTS) " " (storm-conf TOPOLOGY-WORKER-CHILDOPTS)) - "%ID%" - (str port)) + worker-childopts (when-let [s (conf WORKER-CHILDOPTS)] + (substitute-worker-childopts s port)) + topo-worker-childopts (when-let [s (storm-conf TOPOLOGY-WORKER-CHILDOPTS)] + (substitute-worker-childopts s port)) logfilename (str "worker-" port ".log") - command (str "java -server " childopts - " -Djava.library.path=" (conf JAVA-LIBRARY-PATH) - " -Dlogfile.name=" logfilename - " -Dstorm.home=" storm-home - " -Dlogback.configurationFile=" storm-home "/logback/cluster.xml" - " -Dstorm.id=" storm-id - " -Dworker.id=" worker-id - " -Dworker.port=" port - " -cp " classpath " backtype.storm.daemon.worker " - (java.net.URLEncoder/encode storm-id) " " (:assignment-id supervisor) - " " port " " worker-id)] - (log-message "Launching worker with command: " command) + command (concat + ["java" "-server"] + worker-childopts + topo-worker-childopts + [(str "-Djava.library.path=" (conf JAVA-LIBRARY-PATH)) + (str "-Dlogfile.name=" logfilename) + (str "-Dstorm.home=" storm-home) + (str "-Dlogback.configurationFile=" storm-home "/logback/cluster.xml") + (str "-Dstorm.id=" storm-id) + (str "-Dworker.id=" worker-id) + (str "-Dworker.port=" port) + "-cp" classpath + "backtype.storm.daemon.worker" + storm-id + (:assignment-id supervisor) + port + worker-id]) + command (->> command (map str) (filter (complement empty?))) + shell-cmd (->> command + (map #(str \' (clojure.string/escape % {\' "\\'"}) \')) + (clojure.string/join " "))] + (log-message "Launching worker with command: " shell-cmd) (launch-process command :environment {"LD_LIBRARY_PATH" (conf JAVA-LIBRARY-PATH)}) )) diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj index 3055b66bc..0d1f6c692 100644 --- a/storm-core/src/clj/backtype/storm/daemon/worker.clj +++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj @@ -451,4 +451,4 @@ (defn -main [storm-id assignment-id port-str worker-id] (let [conf (read-storm-config)] (validate-distributed-mode! conf) - (mk-worker conf nil (java.net.URLDecoder/decode storm-id) assignment-id (Integer/parseInt port-str) worker-id))) + (mk-worker conf nil storm-id assignment-id (Integer/parseInt port-str) worker-id))) diff --git a/storm-core/src/clj/backtype/storm/timer.clj b/storm-core/src/clj/backtype/storm/timer.clj index f9774b6d1..78b1f1c24 100644 --- a/storm-core/src/clj/backtype/storm/timer.clj +++ b/storm-core/src/clj/backtype/storm/timer.clj @@ -82,7 +82,7 @@ (let [id (uuid) ^PriorityQueue queue (:queue timer)] (locking (:lock timer) - (.add queue [(+ (current-time-millis) (* 1000 (long delay-secs))) afn id]) + (.add queue [(+ (current-time-millis) (secs-to-millis-long delay-secs)) afn id]) ))) (defn schedule-recurring [timer delay-secs recur-secs afn] diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index 63a3053d5..87e79facc 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/storm/ui/core.clj @@ -29,6 +29,7 @@ Nimbus$Client StormTopology GlobalStreamId RebalanceOptions KillOptions]) (:import [java.io File]) + (:import [java.net URLDecoder]) (:require [compojure.route :as route] [compojure.handler :as handler] [ring.util.response :as resp] @@ -485,7 +486,7 @@ :let [disp ((display-map k) k)]] [(link-to (if (= k window) {:class "red"} {}) (url-format "/topology/%s?window=%s" id k) - disp) + (escape-html disp)) (get-in stats [:emitted k]) (get-in stats [:transferred k]) (float-str (get-in stats [:complete-latencies k])) @@ -717,7 +718,7 @@ :let [disp ((display-map k) k)]] [(link-to (if (= k window) {:class "red"} {}) (url-format "/topology/%s/component/%s?window=%s" topology-id id k) - disp) + (escape-html disp)) (get-in stats [:emitted k]) (get-in stats [:transferred k]) (float-str (get-in stats [:complete-latencies k])) @@ -935,7 +936,7 @@ :let [disp ((display-map k) k)]] [(link-to (if (= k window) {:class "red"} {}) (url-format "/topology/%s/component/%s?window=%s" topology-id id k) - disp) + (escape-html disp)) (get-in stats [:emitted k]) (get-in stats [:transferred k]) (float-str (get-in stats [:execute-latencies k])) @@ -1016,34 +1017,40 @@ (-> (main-page) ui-template)) (GET "/topology/:id" [:as {cookies :cookies} id & m] - (let [include-sys? (get-include-sys? cookies)] + (let [include-sys? (get-include-sys? cookies) + id (java.net.URLDecoder/decode id)] (try - (-> (topology-page id (:window m) include-sys?) + (-> (topology-page (URLDecoder/decode id) (:window m) include-sys?) (concat [(mk-system-toggle-button include-sys?)]) ui-template) (catch Exception e (resp/redirect "/"))))) (GET "/topology/:id/component/:component" [:as {cookies :cookies} id component & m] - (let [include-sys? (get-include-sys? cookies)] + (let [include-sys? (get-include-sys? cookies) + id (java.net.URLDecoder/decode id) + component (java.net.URLDecoder/decode component)] (-> (component-page id component (:window m) include-sys?) (concat [(mk-system-toggle-button include-sys?)]) ui-template))) (POST "/topology/:id/activate" [id] (with-nimbus nimbus - (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) + (let [id (java.net.URLDecoder/decode id) + tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg)] (.activate nimbus name) (log-message "Activating topology '" name "'"))) (resp/redirect (str "/topology/" id))) (POST "/topology/:id/deactivate" [id] (with-nimbus nimbus - (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) + (let [id (java.net.URLDecoder/decode id) + tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg)] (.deactivate nimbus name) (log-message "Deactivating topology '" name "'"))) (resp/redirect (str "/topology/" id))) (POST "/topology/:id/rebalance/:wait-time" [id wait-time] (with-nimbus nimbus - (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) + (let [id (java.net.URLDecoder/decode id) + tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg) options (RebalanceOptions.)] (.set_wait_secs options (Integer/parseInt wait-time)) @@ -1052,7 +1059,8 @@ (resp/redirect (str "/topology/" id))) (POST "/topology/:id/kill/:wait-time" [id wait-time] (with-nimbus nimbus - (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id) + (let [id (java.net.URLDecoder/decode id) + tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg) options (KillOptions.)] (.set_wait_secs options (Integer/parseInt wait-time)) diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj index 2ab30c7e2..6c86f83fb 100644 --- a/storm-core/src/clj/backtype/storm/util.clj +++ b/storm-core/src/clj/backtype/storm/util.clj @@ -220,6 +220,9 @@ (defn current-time-millis [] (Time/currentTimeMillis)) +(defn secs-to-millis-long [secs] + (long (* (long 1000) secs))) + (defn clojurify-structure [s] (prewalk (fn [x] (cond (instance? Map x) (into {} x) @@ -395,9 +398,7 @@ )) (defnk launch-process [command :environment {}] - (let [command (->> (seq (.split command " ")) - (filter (complement empty?))) - builder (ProcessBuilder. command) + (let [builder (ProcessBuilder. command) process-env (.environment builder)] (doseq [[k v] environment] (.put process-env k v)) @@ -756,7 +757,7 @@ (defn zip-contains-dir? [zipfile target] (let [entries (->> zipfile (ZipFile.) .entries enumeration-seq (map (memfn getName)))] - (some? #(.startsWith % (str target file-path-separator)) entries) + (some? #(.startsWith % (str target "/")) entries) )) (defn url-encode [s] diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj index 617e69b32..5e70cb1de 100644 --- a/storm-core/src/clj/backtype/storm/zookeeper.clj +++ b/storm-core/src/clj/backtype/storm/zookeeper.clj @@ -21,7 +21,7 @@ ZooDefs ZooDefs$Ids CreateMode WatchedEvent Watcher$Event Watcher$Event$KeeperState Watcher$Event$EventType KeeperException$NodeExistsException]) (:import [org.apache.zookeeper.data Stat]) - (:import [org.apache.zookeeper.server ZooKeeperServer NIOServerCnxn$Factory]) + (:import [org.apache.zookeeper.server ZooKeeperServer NIOServerCnxnFactory]) (:import [java.net InetSocketAddress BindException]) (:import [java.io File]) (:import [backtype.storm.utils Utils ZookeeperAuthInfo]) @@ -156,7 +156,7 @@ (let [localfile (File. localdir) zk (ZooKeeperServer. localfile localfile 2000) [retport factory] (loop [retport (if port port 2000)] - (if-let [factory-tmp (try-cause (NIOServerCnxn$Factory. (InetSocketAddress. retport)) + (if-let [factory-tmp (try-cause (doto (NIOServerCnxnFactory.) (.configure (InetSocketAddress. retport) 0)) (catch BindException e (when (> (inc retport) (if port port 65535)) (throw (RuntimeException. "No port is available to launch an inprocess zookeeper.")))))] diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 281ae525d..d99325c1f 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -43,6 +43,13 @@ * Spouts.

*/ public class Config extends HashMap { + /** + * The serializer for communication between shell components and non-JVM + * processes + */ + public static final String STORM_MULTILANG_SERIALIZER = "storm.multilang.serializer"; + public static final Object STORM_MULTILANG_SERIALIZER_SCHEMA = String.class; + /** * The transporter for communication among Storm tasks */ @@ -459,8 +466,7 @@ public class Config extends HashMap { * with an identifier for this worker. */ public static final String WORKER_CHILDOPTS = "worker.childopts"; - public static final Object WORKER_CHILDOPTS_SCHEMA = String.class; - + public static final Object WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator; /** * How often this worker should heartbeat to the supervisor. @@ -662,7 +668,7 @@ public class Config extends HashMap { * Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS. */ public static final String TOPOLOGY_WORKER_CHILDOPTS="topology.worker.childopts"; - public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = String.class; + public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator; /** * This config is available for TransactionalSpouts, and contains the id ( a String) for diff --git a/storm-core/src/jvm/backtype/storm/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/ConfigValidation.java index 15ef6ba12..3accb820e 100644 --- a/storm-core/src/jvm/backtype/storm/ConfigValidation.java +++ b/storm-core/src/jvm/backtype/storm/ConfigValidation.java @@ -135,4 +135,21 @@ public void validateField(String name, Object o) throws IllegalArgumentException "Field " + name + " must be an Iterable containing only Strings or Maps of Strings"); } }; + + /** + * Validates a String or a list of Strings + */ + public static Object StringOrStringListValidator = new FieldValidator() { + + private FieldValidator fv = FieldListValidatorFactory(String.class); + + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null || o instanceof String) { + // A null value or a String value is acceptable + return; + } + this.fv.validateField(name, o); + } + }; } diff --git a/storm-core/src/jvm/backtype/storm/StormSubmitter.java b/storm-core/src/jvm/backtype/storm/StormSubmitter.java index 80734568b..2fd0ce88b 100644 --- a/storm-core/src/jvm/backtype/storm/StormSubmitter.java +++ b/storm-core/src/jvm/backtype/storm/StormSubmitter.java @@ -17,17 +17,25 @@ */ package backtype.storm; -import backtype.storm.generated.*; -import backtype.storm.utils.BufferFileInputStream; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import org.apache.thrift.TException; import org.json.simple.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.AlreadyAliveException; +import backtype.storm.generated.ClusterSummary; +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.Nimbus; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.SubmitOptions; +import backtype.storm.generated.TopologySummary; +import backtype.storm.utils.BufferFileInputStream; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; /** * Use this class to submit topologies to run on the Storm cluster. You should run your program @@ -37,6 +45,8 @@ public class StormSubmitter { public static Logger LOG = LoggerFactory.getLogger(StormSubmitter.class); + private static final int THRIFT_CHUNK_SIZE_BYTES = 307200; + private static Nimbus.Iface localNimbus = null; public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) { @@ -151,7 +161,7 @@ public static String submitJar(Map conf, String localJar) { try { String uploadLocation = client.getClient().beginFileUpload(); LOG.info("Uploading topology jar " + localJar + " to assigned location: " + uploadLocation); - BufferFileInputStream is = new BufferFileInputStream(localJar); + BufferFileInputStream is = new BufferFileInputStream(localJar, THRIFT_CHUNK_SIZE_BYTES); while(true) { byte[] toSubmit = is.read(); if(toSubmit.length==0) break; diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index d765e7105..6996b49cf 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -21,17 +21,21 @@ import backtype.storm.messaging.IConnection; import backtype.storm.messaging.TaskMessage; import backtype.storm.utils.Utils; + import org.jboss.netty.bootstrap.ClientBootstrap; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; import java.util.Map; +import java.util.Timer; +import java.util.TimerTask; import java.util.Random; -import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -39,38 +43,37 @@ class Client implements IConnection { private static final Logger LOG = LoggerFactory.getLogger(Client.class); + private static final Timer TIMER = new Timer("netty-client-timer", true); + private final int max_retries; - private final int base_sleep_ms; - private final int max_sleep_ms; + private final long base_sleep_ms; + private final long max_sleep_ms; private LinkedBlockingQueue message_queue; //entry should either be TaskMessage or ControlMessage private AtomicReference channelRef; private final ClientBootstrap bootstrap; - private InetSocketAddress remote_addr; + InetSocketAddress remote_addr; private AtomicInteger retries; private final Random random = new Random(); private final ChannelFactory factory; private final int buffer_size; private final AtomicBoolean being_closed; + private boolean wait_for_requests; @SuppressWarnings("rawtypes") - Client(Map storm_conf, String host, int port) { + Client(Map storm_conf, ChannelFactory factory, String host, int port) { + this.factory = factory; message_queue = new LinkedBlockingQueue(); retries = new AtomicInteger(0); channelRef = new AtomicReference(null); being_closed = new AtomicBoolean(false); + wait_for_requests = false; // Configure buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); max_retries = Math.min(30, Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES))); base_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); max_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); - int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS)); - if (maxWorkers > 0) { - factory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), maxWorkers); - } else { - factory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()); - } bootstrap = new ClientBootstrap(factory); bootstrap.setOption("tcpNoDelay", true); bootstrap.setOption("sendBufferSize", buffer_size); @@ -88,29 +91,37 @@ class Client implements IConnection { * We will retry connection with exponential back-off policy */ void reconnect() { - try { - int tried_count = retries.incrementAndGet(); - if (tried_count <= max_retries) { - Thread.sleep(getSleepTimeMs()); - LOG.info("Reconnect ... [{}]", tried_count); - bootstrap.connect(remote_addr); - LOG.debug("connection started..."); - } else { - LOG.warn("Remote address is not reachable. We will close this client."); - close(); - } - } catch (InterruptedException e) { - LOG.warn("connection failed", e); + close_n_release(); + + //reconnect only if it's not being closed + if (being_closed.get()) return; + + final int tried_count = retries.incrementAndGet(); + if (tried_count <= max_retries) { + long sleep = getSleepTimeMs(); + LOG.info("Waiting {} ms before trying connection to {}", sleep, remote_addr); + TIMER.schedule(new TimerTask() { + @Override + public void run() { + LOG.info("Reconnect ... [{}] to {}", tried_count, remote_addr); + bootstrap.connect(remote_addr); + }}, sleep); + } else { + LOG.warn(remote_addr+" is not reachable. We will close this client."); + close(); } } /** * # of milliseconds to wait per exponential back-off policy */ - private int getSleepTimeMs() + private long getSleepTimeMs() { + if (retries.get() > 30) { + return max_sleep_ms; + } int backoff = 1 << retries.get(); - int sleepMs = base_sleep_ms * Math.max(1, random.nextInt(backoff)); + long sleepMs = base_sleep_ms * Math.max(1, random.nextInt(backoff)); if ( sleepMs > max_sleep_ms ) sleepMs = max_sleep_ms; return sleepMs; @@ -127,36 +138,94 @@ public void send(int task, byte[] message) { try { message_queue.put(new TaskMessage(task, message)); + + //resume delivery if it is waiting for requests + tryDeliverMessages(true); } catch (InterruptedException e) { throw new RuntimeException(e); } } + /** + * Retrieve messages from queue, and delivery to server if any + */ + synchronized void tryDeliverMessages(boolean only_if_waiting) throws InterruptedException { + //just skip if delivery only if waiting, and we are not waiting currently + if (only_if_waiting && !wait_for_requests) return; + + //make sure that channel was not closed + Channel channel = channelRef.get(); + if (channel == null) return; + if (!channel.isOpen()) { + LOG.info("Channel to {} is no longer open.",remote_addr); + //The channel is not open yet. Reconnect? + reconnect(); + return; + } + + final MessageBatch requests = tryTakeMessages(); + if (requests==null) { + wait_for_requests = true; + return; + } + + //if channel is being closed and we have no outstanding messages, let's close the channel + if (requests.isEmpty() && being_closed.get()) { + close_n_release(); + return; + } + + //we are busily delivering messages, and will check queue upon response. + //When send() is called by senders, we should not thus call tryDeliverMessages(). + wait_for_requests = false; + + //write request into socket channel + ChannelFuture future = channel.write(requests); + future.addListener(new ChannelFutureListener() { + public void operationComplete(ChannelFuture future) + throws Exception { + if (!future.isSuccess()) { + LOG.info("failed to send "+requests.size()+" requests to "+remote_addr, future.getCause()); + reconnect(); + } else { + LOG.debug("{} request(s) sent", requests.size()); + + //Now that our requests have been sent, channel could be closed if needed + if (being_closed.get()) + close_n_release(); + } + } + }); + } + /** * Take all enqueued messages from queue - * @return + * @return batch of messages * @throws InterruptedException + * + * synchronized ... ensure that messages are delivered in the same order + * as they are added into queue */ - MessageBatch takeMessages() throws InterruptedException { + private MessageBatch tryTakeMessages() throws InterruptedException { //1st message - MessageBatch batch = new MessageBatch(buffer_size); - Object msg = message_queue.take(); - batch.add(msg); + Object msg = message_queue.poll(); + if (msg == null) return null; + MessageBatch batch = new MessageBatch(buffer_size); //we will discard any message after CLOSE - if (msg==ControlMessage.CLOSE_MESSAGE) + if (msg == ControlMessage.CLOSE_MESSAGE) { + LOG.info("Connection to {} is being closed", remote_addr); + being_closed.set(true); return batch; + } - while (!batch.isFull()) { - //peek the next message - msg = message_queue.peek(); - //no more messages - if (msg == null) break; - - //we will discard any message after CLOSE - if (msg==ControlMessage.CLOSE_MESSAGE) { + batch.add((TaskMessage)msg); + while (!batch.isFull() && ((msg = message_queue.peek())!=null)) { + //Is it a CLOSE message? + if (msg == ControlMessage.CLOSE_MESSAGE) { message_queue.take(); - batch.add(msg); + LOG.info("Connection to {} is being closed", remote_addr); + being_closed.set(true); break; } @@ -176,31 +245,29 @@ MessageBatch takeMessages() throws InterruptedException { * * We will send all existing requests, and then invoke close_n_release() method */ - public synchronized void close() { - if (!being_closed.get()) { - //enqueue a CLOSE message so that shutdown() will be invoked - try { - message_queue.put(ControlMessage.CLOSE_MESSAGE); - being_closed.set(true); - } catch (InterruptedException e) { - close_n_release(); - } + public void close() { + //enqueue a CLOSE message so that shutdown() will be invoked + try { + message_queue.put(ControlMessage.CLOSE_MESSAGE); + + //resume delivery if it is waiting for requests + tryDeliverMessages(true); + } catch (InterruptedException e) { + LOG.info("Interrupted Connection to {} is being closed", remote_addr); + being_closed.set(true); + close_n_release(); } } /** * close_n_release() is invoked after all messages have been sent. */ - void close_n_release() { - if (channelRef.get() != null) - channelRef.get().close().awaitUninterruptibly(); - - //we need to release resources - new Thread(new Runnable() { - @Override - public void run() { - factory.releaseExternalResources(); - }}).start(); + synchronized void close_n_release() { + if (channelRef.get() != null) { + channelRef.get().close(); + LOG.debug("channel {} closed",remote_addr); + setChannel(null); + } } public TaskMessage recv(int flags) { @@ -208,6 +275,10 @@ public TaskMessage recv(int flags) { } void setChannel(Channel channel) { + if (channel != null && channel.isOpen()) { + //Assume the most recent connection attempt was successful. + retries.set(0); + } channelRef.set(channel); //reset retries if (channel != null) @@ -215,7 +286,3 @@ void setChannel(Channel channel) { } } - - - - diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java index 3e09dd1c8..80b4443e1 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java @@ -17,8 +17,16 @@ */ package backtype.storm.messaging.netty; +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; + +import java.util.concurrent.Executors; +import java.util.Map; +import java.util.Vector; + +import backtype.storm.Config; import backtype.storm.messaging.IConnection; import backtype.storm.messaging.IContext; +import backtype.storm.utils.Utils; import java.util.Map; import java.util.Vector; @@ -27,14 +35,25 @@ public class Context implements IContext { @SuppressWarnings("rawtypes") private Map storm_conf; private volatile Vector connections; - + private NioClientSocketChannelFactory clientChannelFactory; + /** * initialization per Storm configuration */ @SuppressWarnings("rawtypes") public void prepare(Map storm_conf) { - this.storm_conf = storm_conf; - connections = new Vector(); + this.storm_conf = storm_conf; + connections = new Vector(); + + //each context will have a single client channel factory + int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS)); + if (maxWorkers > 0) { + clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), + Executors.newCachedThreadPool(), maxWorkers); + } else { + clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), + Executors.newCachedThreadPool()); + } } /** @@ -50,7 +69,7 @@ public IConnection bind(String storm_id, int port) { * establish a connection to a remote server */ public IConnection connect(String storm_id, String host, int port) { - IConnection client = new Client(storm_conf, host, port); + IConnection client = new Client(storm_conf, clientChannelFactory, host, port); connections.add(client); return client; } @@ -63,5 +82,8 @@ public void term() { conn.close(); } connections = null; + + //we need to release resources associated with client channel factory + clientChannelFactory.releaseExternalResources(); } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java index 9d287e4b5..cd8d4e378 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java @@ -26,56 +26,22 @@ class MessageBatch { private int buffer_size; - private ArrayList msgs; + private ArrayList msgs; private int encoded_length; MessageBatch(int buffer_size) { this.buffer_size = buffer_size; - msgs = new ArrayList(); + msgs = new ArrayList(); encoded_length = ControlMessage.EOB_MESSAGE.encodeLength(); } - void add(Object obj) { + void add(TaskMessage obj) { if (obj == null) throw new RuntimeException("null object forbidded in message batch"); - if (obj instanceof TaskMessage) { - TaskMessage msg = (TaskMessage)obj; - msgs.add(msg); - encoded_length += msgEncodeLength(msg); - return; - } - - if (obj instanceof ControlMessage) { - ControlMessage msg = (ControlMessage)obj; - msgs.add(msg); - encoded_length += msg.encodeLength(); - return; - } - - throw new RuntimeException("Unsuppoted object type "+obj.getClass().getName()); - } - - void remove(Object obj) { - if (obj == null) return; - - if (obj instanceof TaskMessage) { - TaskMessage msg = (TaskMessage)obj; - msgs.remove(msg); - encoded_length -= msgEncodeLength(msg); - return; - } - - if (obj instanceof ControlMessage) { - ControlMessage msg = (ControlMessage)obj; - msgs.remove(msg); - encoded_length -= msg.encodeLength(); - return; - } - } - - Object get(int index) { - return msgs.get(index); + TaskMessage msg = (TaskMessage)obj; + msgs.add(msg); + encoded_length += msgEncodeLength(msg); } /** @@ -129,12 +95,9 @@ int size() { ChannelBuffer buffer() throws Exception { ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encoded_length)); - for (Object msg : msgs) - if (msg instanceof TaskMessage) - writeTaskMessage(bout, (TaskMessage)msg); - else - ((ControlMessage)msg).write(bout); - + for (TaskMessage msg : msgs) + writeTaskMessage(bout, msg); + //add a END_OF_BATCH indicator ControlMessage.EOB_MESSAGE.write(bout); diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java index ad811b062..83e418767 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java @@ -123,7 +123,7 @@ protected void closeChannel(Channel channel) { * close all channels, and release resources */ public synchronized void close() { - if (allChannels != null) { + if (allChannels != null) { allChannels.close().awaitUninterruptibly(); factory.releaseExternalResources(); allChannels = null; diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java index 65c36a7c5..43a8c3912 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java @@ -17,7 +17,15 @@ */ package backtype.storm.messaging.netty; -import org.jboss.netty.channel.*; +import java.net.ConnectException; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelStateEvent; +import org.jboss.netty.channel.ExceptionEvent; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelUpstreamHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,12 +35,10 @@ public class StormClientHandler extends SimpleChannelUpstreamHandler { private static final Logger LOG = LoggerFactory.getLogger(StormClientHandler.class); private Client client; - private AtomicBoolean being_closed; - long start_time; + long start_time; StormClientHandler(Client client) { this.client = client; - being_closed = new AtomicBoolean(false); start_time = System.currentTimeMillis(); } @@ -41,13 +47,14 @@ public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent event) //register the newly established channel Channel channel = event.getChannel(); client.setChannel(channel); - LOG.debug("connection established to a remote host"); + LOG.info("connection established from "+channel.getLocalAddress()+" to "+channel.getRemoteAddress()); - //send next request + //send next batch of requests if any try { - sendRequests(channel, client.takeMessages()); - } catch (InterruptedException e) { - channel.close(); + client.tryDeliverMessages(false); + } catch (Exception ex) { + LOG.info("exception when sending messages:", ex.getMessage()); + client.reconnect(); } } @@ -60,62 +67,21 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) { if (msg==ControlMessage.FAILURE_RESPONSE) LOG.info("failure response:{}", msg); - //send next request - Channel channel = event.getChannel(); + //send next batch of requests if any try { - sendRequests(channel, client.takeMessages()); - } catch (InterruptedException e) { - channel.close(); - } - } - - /** - * Retrieve a request from message queue, and send to server - * @param channel - */ - private void sendRequests(Channel channel, final MessageBatch requests) { - if (requests==null || requests.size()==0 || being_closed.get()) return; - - //if task==CLOSE_MESSAGE for our last request, the channel is to be closed - Object last_msg = requests.get(requests.size()-1); - if (last_msg==ControlMessage.CLOSE_MESSAGE) { - being_closed.set(true); - requests.remove(last_msg); - } - - //we may don't need do anything if no requests found - if (requests.isEmpty()) { - if (being_closed.get()) - client.close_n_release(); - return; + client.tryDeliverMessages(false); + } catch (Exception ex) { + LOG.info("exception when sending messages:", ex.getMessage()); + client.reconnect(); } - - //write request into socket channel - ChannelFuture future = channel.write(requests); - future.addListener(new ChannelFutureListener() { - public void operationComplete(ChannelFuture future) - throws Exception { - if (!future.isSuccess()) { - LOG.info("failed to send requests:", future.getCause()); - future.getChannel().close(); - } else { - LOG.debug("{} request(s) sent", requests.size()); - } - if (being_closed.get()) - client.close_n_release(); - } - }); } @Override public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) { Throwable cause = event.getCause(); if (!(cause instanceof ConnectException)) { - LOG.info("Connection failed:", cause); - } - if (!being_closed.get()) { - client.setChannel(null); - client.reconnect(); + LOG.info("Connection to "+client.remote_addr+" failed:", cause); } + client.reconnect(); } } diff --git a/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java b/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java new file mode 100644 index 000000000..1d6bd1d9e --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java @@ -0,0 +1,63 @@ +package backtype.storm.multilang; + +import java.util.List; + +/** + * BoltMsg is an object that represents the data sent from a shell component to + * a bolt process that implements a multi-language protocol. It is the union of + * all data types that a bolt can receive from Storm. + * + *

+ * BoltMsgs are objects sent to the ISerializer interface, for serialization + * according to the wire protocol implemented by the serializer. The BoltMsg + * class allows for a decoupling between the serialized representation of the + * data and the data itself. + *

+ */ +public class BoltMsg { + private String id; + private String comp; + private String stream; + private long task; + private List tuple; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getComp() { + return comp; + } + + public void setComp(String comp) { + this.comp = comp; + } + + public String getStream() { + return stream; + } + + public void setStream(String stream) { + this.stream = stream; + } + + public long getTask() { + return task; + } + + public void setTask(long task) { + this.task = task; + } + + public List getTuple() { + return tuple; + } + + public void setTuple(List tuple) { + this.tuple = tuple; + } +} diff --git a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java new file mode 100644 index 000000000..a6ccfd432 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java @@ -0,0 +1,65 @@ +package backtype.storm.multilang; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +import backtype.storm.task.TopologyContext; + +/** + * The ISerializer interface describes the methods that an object should + * implement to provide serialization and de-serialization capabilities to + * non-JVM language components. + */ +public interface ISerializer extends Serializable { + + /** + * This method sets the input and output streams of the serializer + * + * @param processIn output stream to non-JVM component + * @param processOut input stream from non-JVM component + */ + void initialize(OutputStream processIn, InputStream processOut); + + /** + * This method transmits the Storm config to the non-JVM process and + * receives its pid. + * + * @param conf storm configuration + * @param context topology context + * @return process pid + */ + Number connect(Map conf, TopologyContext context) throws IOException, + NoOutputException; + + /** + * This method receives a shell message from the non-JVM process + * + * @return shell message + */ + ShellMsg readShellMsg() throws IOException, NoOutputException; + + /** + * This method sends a bolt message to a non-JVM bolt process + * + * @param msg bolt message + */ + void writeBoltMsg(BoltMsg msg) throws IOException; + + /** + * This method sends a spout message to a non-JVM spout process + * + * @param msg spout message + */ + void writeSpoutMsg(SpoutMsg msg) throws IOException; + + /** + * This method sends a list of task IDs to a non-JVM bolt process + * + * @param taskIds list of task IDs + */ + void writeTaskIds(List taskIds) throws IOException; +} diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java new file mode 100644 index 000000000..70d8e5b79 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java @@ -0,0 +1,164 @@ +package backtype.storm.multilang; + +import java.io.BufferedReader; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.Utils; + +/** + * JsonSerializer implements the JSON multilang protocol. + */ +public class JsonSerializer implements ISerializer { + private DataOutputStream processIn; + private BufferedReader processOut; + + public void initialize(OutputStream processIn, InputStream processOut) { + this.processIn = new DataOutputStream(processIn); + this.processOut = new BufferedReader(new InputStreamReader(processOut)); + } + + public Number connect(Map conf, TopologyContext context) + throws IOException, NoOutputException { + JSONObject setupInfo = new JSONObject(); + setupInfo.put("pidDir", context.getPIDDir()); + setupInfo.put("conf", conf); + setupInfo.put("context", context); + writeMessage(setupInfo); + + Number pid = (Number) ((JSONObject) readMessage()).get("pid"); + return pid; + } + + public void writeBoltMsg(BoltMsg boltMsg) throws IOException { + JSONObject obj = new JSONObject(); + obj.put("id", boltMsg.getId()); + obj.put("comp", boltMsg.getComp()); + obj.put("stream", boltMsg.getStream()); + obj.put("task", boltMsg.getTask()); + obj.put("tuple", boltMsg.getTuple()); + writeMessage(obj); + } + + public void writeSpoutMsg(SpoutMsg msg) throws IOException { + JSONObject obj = new JSONObject(); + obj.put("command", msg.getCommand()); + obj.put("id", msg.getId()); + writeMessage(obj); + } + + public void writeTaskIds(List taskIds) throws IOException { + writeMessage(taskIds); + } + + private void writeMessage(Object msg) throws IOException { + writeString(JSONValue.toJSONString(msg)); + } + + private void writeString(String str) throws IOException { + byte[] strBytes = str.getBytes("UTF-8"); + processIn.write(strBytes, 0, strBytes.length); + processIn.writeBytes("\nend\n"); + processIn.flush(); + } + + public ShellMsg readShellMsg() throws IOException, NoOutputException { + JSONObject msg = (JSONObject) readMessage(); + ShellMsg shellMsg = new ShellMsg(); + + String command = (String) msg.get("command"); + shellMsg.setCommand(command); + + String id = (String) msg.get("id"); + shellMsg.setId(id); + + String log = (String) msg.get("msg"); + shellMsg.setMsg(log); + + String stream = (String) msg.get("stream"); + if (stream == null) + stream = Utils.DEFAULT_STREAM_ID; + shellMsg.setStream(stream); + + Object taskObj = msg.get("task"); + if (taskObj != null) { + shellMsg.setTask((Long) taskObj); + } else { + shellMsg.setTask(0); + } + + Object need_task_ids = msg.get("need_task_ids"); + if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) { + shellMsg.setNeedTaskIds(true); + } else { + shellMsg.setNeedTaskIds(false); + } + + shellMsg.setTuple((List) msg.get("tuple")); + + List anchors = new ArrayList(); + Object anchorObj = msg.get("anchors"); + if (anchorObj != null) { + if (anchorObj instanceof String) { + anchorObj = Arrays.asList(anchorObj); + } + for (Object o : (List) anchorObj) { + shellMsg.addAnchor((String) o); + } + } + + return shellMsg; + } + + private Object readMessage() throws IOException, NoOutputException { + String string = readString(); + Object msg = JSONValue.parse(string); + if (msg != null) { + return msg; + } else { + throw new IOException("unable to parse: " + string); + } + } + + private String readString() throws IOException, NoOutputException { + StringBuilder line = new StringBuilder(); + + // synchronized (processOut) { + while (true) { + String subline = processOut.readLine(); + if (subline == null) { + StringBuilder errorMessage = new StringBuilder(); + errorMessage.append("Pipe to subprocess seems to be broken!"); + if (line.length() == 0) { + errorMessage.append(" No output read.\n"); + } else { + errorMessage.append(" Currently read output: " + + line.toString() + "\n"); + } + errorMessage.append("Serializer Exception:\n"); + throw new NoOutputException(errorMessage.toString()); + } + if (subline.equals("end")) { + break; + } + if (line.length() != 0) { + line.append("\n"); + } + line.append(subline); + } + // } + return line.toString(); + } +} diff --git a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java new file mode 100644 index 000000000..768f65fe0 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java @@ -0,0 +1,23 @@ +package backtype.storm.multilang; + +/** + * A NoOutputException states that no data has been received from the connected + * non-JVM process. + */ +public class NoOutputException extends Exception { + public NoOutputException() { + super(); + } + + public NoOutputException(String message) { + super(message); + } + + public NoOutputException(String message, Throwable cause) { + super(message, cause); + } + + public NoOutputException(Throwable cause) { + super(cause); + } +} diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java new file mode 100644 index 000000000..4381c07db --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java @@ -0,0 +1,105 @@ +package backtype.storm.multilang; + +import java.util.ArrayList; +import java.util.List; + +/** + * ShellMsg is an object that represents the data sent to a shell component from + * a process that implements a multi-language protocol. It is the union of all + * data types that a component can send to Storm. + * + *

+ * ShellMsgs are objects received from the ISerializer interface, after the + * serializer has deserialized the data from the underlying wire protocol. The + * ShellMsg class allows for a decoupling between the serialized representation + * of the data and the data itself. + *

+ */ +public class ShellMsg { + private String command; + private String id; + private List anchors; + private String stream; + private long task; + private String msg; + private List tuple; + private boolean needTaskIds; + + public String getCommand() { + return command; + } + + public void setCommand(String command) { + this.command = command; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public List getAnchors() { + return anchors; + } + + public void setAnchors(List anchors) { + this.anchors = anchors; + } + + public void addAnchor(String anchor) { + if (anchors == null) { + anchors = new ArrayList(); + } + this.anchors.add(anchor); + } + + public String getStream() { + return stream; + } + + public void setStream(String stream) { + this.stream = stream; + } + + public long getTask() { + return task; + } + + public void setTask(long task) { + this.task = task; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + public List getTuple() { + return tuple; + } + + public void setTuple(List tuple) { + this.tuple = tuple; + } + + public void addTuple(Object tuple) { + if (this.tuple == null) { + this.tuple = new ArrayList(); + } + this.tuple.add(tuple); + } + + public boolean areTaskIdsNeeded() { + return needTaskIds; + } + + public void setNeedTaskIds(boolean needTaskIds) { + this.needTaskIds = needTaskIds; + } +} diff --git a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java new file mode 100644 index 000000000..9b5ffce91 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java @@ -0,0 +1,34 @@ +package backtype.storm.multilang; + +/** + * SpoutMsg is an object that represents the data sent from a shell spout to a + * process that implements a multi-language spout. The SpoutMsg is used to send + * a "next", "ack" or "fail" message to a spout. + * + *

+ * Spout messages are objects sent to the ISerializer interface, for + * serialization according to the wire protocol implemented by the serializer. + * The SpoutMsg class allows for a decoupling between the serialized + * representation of the data and the data itself. + *

+ */ +public class SpoutMsg { + private String command; + private String id; + + public String getCommand() { + return command; + } + + public void setCommand(String command) { + this.command = command; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } +} diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java index 67cb66fb3..b28348c09 100644 --- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java +++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java @@ -18,15 +18,15 @@ package backtype.storm.spout; import backtype.storm.generated.ShellComponent; +import backtype.storm.multilang.ShellMsg; +import backtype.storm.multilang.SpoutMsg; import backtype.storm.task.TopologyContext; import backtype.storm.utils.ShellProcess; -import backtype.storm.utils.Utils; import java.util.Map; import java.util.List; import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.json.simple.JSONObject; public class ShellSpout implements ISpout { @@ -35,91 +35,84 @@ public class ShellSpout implements ISpout { private SpoutOutputCollector _collector; private String[] _command; private ShellProcess _process; + private SpoutMsg spoutMsg; public ShellSpout(ShellComponent component) { this(component.get_execution_command(), component.get_script()); } - + public ShellSpout(String... command) { _command = command; } - + public void open(Map stormConf, TopologyContext context, SpoutOutputCollector collector) { - _process = new ShellProcess(_command); _collector = collector; - try { - Number subpid = _process.launch(stormConf, context); - LOG.info("Launched subprocess with pid " + subpid); - } catch (IOException e) { - throw new RuntimeException("Error when launching multilang subprocess\n" + _process.getErrorsString(), e); - } + _process = new ShellProcess(_command); + + Number subpid = _process.launch(stormConf, context); + LOG.info("Launched subprocess with pid " + subpid); } public void close() { _process.destroy(); } - private JSONObject _next; public void nextTuple() { - if (_next == null) { - _next = new JSONObject(); - _next.put("command", "next"); + if (spoutMsg == null) { + spoutMsg = new SpoutMsg(); } - - querySubprocess(_next); + spoutMsg.setCommand("next"); + spoutMsg.setId(""); + querySubprocess(); } - private JSONObject _ack; public void ack(Object msgId) { - if (_ack == null) { - _ack = new JSONObject(); - _ack.put("command", "ack"); + if (spoutMsg == null) { + spoutMsg = new SpoutMsg(); } - - _ack.put("id", msgId); - querySubprocess(_ack); + spoutMsg.setCommand("ack"); + spoutMsg.setId(msgId.toString()); + querySubprocess(); } - private JSONObject _fail; public void fail(Object msgId) { - if (_fail == null) { - _fail = new JSONObject(); - _fail.put("command", "fail"); + if (spoutMsg == null) { + spoutMsg = new SpoutMsg(); } - - _fail.put("id", msgId); - querySubprocess(_fail); + spoutMsg.setCommand("fail"); + spoutMsg.setId(msgId.toString()); + querySubprocess(); } - private void querySubprocess(Object query) { + private void querySubprocess() { try { - _process.writeMessage(query); + _process.writeSpoutMsg(spoutMsg); while (true) { - JSONObject action = _process.readMessage(); - String command = (String) action.get("command"); + ShellMsg shellMsg = _process.readShellMsg(); + String command = shellMsg.getCommand(); if (command.equals("sync")) { return; } else if (command.equals("log")) { - String msg = (String) action.get("msg"); + String msg = shellMsg.getMsg(); LOG.info("Shell msg: " + msg); } else if (command.equals("emit")) { - String stream = (String) action.get("stream"); - if (stream == null) stream = Utils.DEFAULT_STREAM_ID; - Long task = (Long) action.get("task"); - List tuple = (List) action.get("tuple"); - Object messageId = (Object) action.get("id"); - if (task == null) { + String stream = shellMsg.getStream(); + Long task = shellMsg.getTask(); + List tuple = shellMsg.getTuple(); + Object messageId = shellMsg.getId(); + if (task == 0) { List outtasks = _collector.emit(stream, tuple, messageId); - Object need_task_ids = action.get("need_task_ids"); - if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) { - _process.writeMessage(outtasks); + if (shellMsg.areTaskIdsNeeded()) { + _process.writeTaskIds(outtasks); } } else { - _collector.emitDirect((int)task.longValue(), stream, tuple, messageId); + _collector.emitDirect((int) task.longValue(), stream, tuple, messageId); } + } else { + throw new RuntimeException("Unknown command received: " + command); } } } catch (IOException e) { diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java index 118e90ee6..27e55bbe7 100644 --- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java +++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java @@ -19,22 +19,20 @@ import backtype.storm.Config; import backtype.storm.generated.ShellComponent; -import backtype.storm.tuple.MessageId; import backtype.storm.tuple.Tuple; -import backtype.storm.utils.Utils; import backtype.storm.utils.ShellProcess; -import java.io.IOException; +import backtype.storm.multilang.BoltMsg; +import backtype.storm.multilang.ShellMsg; + import java.util.ArrayList; -import java.util.Arrays; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; import static java.util.concurrent.TimeUnit.SECONDS; -import java.util.List; import java.util.Map; import java.util.Random; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.json.simple.JSONObject; /** * A bolt that shells out to another process to process tuples. ShellBolt @@ -75,7 +73,7 @@ public class ShellBolt implements IBolt { private volatile Throwable _exception; private LinkedBlockingQueue _pendingWrites = new LinkedBlockingQueue(); private Random _rand; - + private Thread _readerThread; private Thread _writerThread; @@ -94,39 +92,32 @@ public void prepare(Map stormConf, TopologyContext context, this._pendingWrites = new LinkedBlockingQueue(((Number)maxPending).intValue()); } _rand = new Random(); - _process = new ShellProcess(_command); _collector = collector; + _process = new ShellProcess(_command); - try { - //subprocesses must send their pid first thing - Number subpid = _process.launch(stormConf, context); - LOG.info("Launched subprocess with pid " + subpid); - } catch (IOException e) { - throw new RuntimeException("Error when launching multilang subprocess\n" + _process.getErrorsString(), e); - } + //subprocesses must send their pid first thing + Number subpid = _process.launch(stormConf, context); + LOG.info("Launched subprocess with pid " + subpid); // reader _readerThread = new Thread(new Runnable() { public void run() { while (_running) { try { - JSONObject action = _process.readMessage(); - if (action == null) { - // ignore sync - } + ShellMsg shellMsg = _process.readShellMsg(); - String command = (String) action.get("command"); + String command = shellMsg.getCommand(); if(command.equals("ack")) { - handleAck(action); + handleAck(shellMsg.getId()); } else if (command.equals("fail")) { - handleFail(action); + handleFail(shellMsg.getId()); } else if (command.equals("error")) { - handleError(action); + handleError(shellMsg.getMsg()); } else if (command.equals("log")) { - String msg = (String) action.get("msg"); + String msg = shellMsg.getMsg(); LOG.info("Shell msg: " + msg); } else if (command.equals("emit")) { - handleEmit(action); + handleEmit(shellMsg); } } catch (InterruptedException e) { } catch (Throwable t) { @@ -135,7 +126,7 @@ public void run() { } } }); - + _readerThread.start(); _writerThread = new Thread(new Runnable() { @@ -143,11 +134,13 @@ public void run() { while (_running) { try { Object write = _pendingWrites.poll(1, SECONDS); - if (write != null) { - _process.writeMessage(write); + if (write instanceof BoltMsg) { + _process.writeBoltMsg((BoltMsg)write); + } else if (write instanceof List) { + _process.writeTaskIds((List)write); + } else if (write != null) { + throw new RuntimeException("Unknown class type to write: " + write.getClass().getName()); } - // drain the error stream to avoid dead lock because of full error stream buffer - _process.drainErrorStream(); } catch (InterruptedException e) { } catch (Throwable t) { die(t); @@ -155,7 +148,7 @@ public void run() { } } }); - + _writerThread.start(); } @@ -168,13 +161,14 @@ public void execute(Tuple input) { String genId = Long.toString(_rand.nextLong()); _inputs.put(genId, input); try { - JSONObject obj = new JSONObject(); - obj.put("id", genId); - obj.put("comp", input.getSourceComponent()); - obj.put("stream", input.getSourceStreamId()); - obj.put("task", input.getSourceTask()); - obj.put("tuple", input.getValues()); - _pendingWrites.put(obj); + BoltMsg boltMsg = new BoltMsg(); + boltMsg.setId(genId); + boltMsg.setComp(input.getSourceComponent()); + boltMsg.setStream(input.getSourceStreamId()); + boltMsg.setTask(input.getSourceTask()); + boltMsg.setTuple(input.getValues()); + + _pendingWrites.put(boltMsg); } catch(InterruptedException e) { throw new RuntimeException("Error during multilang processing", e); } @@ -186,8 +180,7 @@ public void cleanup() { _inputs.clear(); } - private void handleAck(Map action) { - String id = (String) action.get("id"); + private void handleAck(String id) { Tuple acked = _inputs.remove(id); if(acked==null) { throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id); @@ -195,8 +188,7 @@ private void handleAck(Map action) { _collector.ack(acked); } - private void handleFail(Map action) { - String id = (String) action.get("id"); + private void handleFail(String id) { Tuple failed = _inputs.remove(id); if(failed==null) { throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id); @@ -204,38 +196,31 @@ private void handleFail(Map action) { _collector.fail(failed); } - private void handleError(Map action) { - String msg = (String) action.get("msg"); + private void handleError(String msg) { _collector.reportError(new Exception("Shell Process Exception: " + msg)); } - private void handleEmit(Map action) throws InterruptedException { - String stream = (String) action.get("stream"); - if(stream==null) stream = Utils.DEFAULT_STREAM_ID; - Long task = (Long) action.get("task"); - List tuple = (List) action.get("tuple"); + private void handleEmit(ShellMsg shellMsg) throws InterruptedException { List anchors = new ArrayList(); - Object anchorObj = action.get("anchors"); - if(anchorObj!=null) { - if(anchorObj instanceof String) { - anchorObj = Arrays.asList(anchorObj); - } - for(Object o: (List) anchorObj) { - Tuple t = _inputs.get((String) o); + List recvAnchors = shellMsg.getAnchors(); + if (recvAnchors != null) { + for (String anchor : recvAnchors) { + Tuple t = _inputs.get(anchor); if (t == null) { - throw new RuntimeException("Anchored onto " + o + " after ack/fail"); + throw new RuntimeException("Anchored onto " + anchor + " after ack/fail"); } anchors.add(t); } } - if(task==null) { - List outtasks = _collector.emit(stream, anchors, tuple); - Object need_task_ids = action.get("need_task_ids"); - if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) { + + if(shellMsg.getTask() == 0) { + List outtasks = _collector.emit(shellMsg.getStream(), anchors, shellMsg.getTuple()); + if (shellMsg.areTaskIdsNeeded()) { _pendingWrites.put(outtasks); } } else { - _collector.emitDirect((int)task.longValue(), stream, anchors, tuple); + _collector.emitDirect((int) shellMsg.getTask(), + shellMsg.getStream(), anchors, shellMsg.getTuple()); } } diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java index 5c3e3c9ac..35a8c9446 100644 --- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java +++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java @@ -17,133 +17,128 @@ */ package backtype.storm.utils; +import backtype.storm.Config; +import backtype.storm.multilang.ISerializer; +import backtype.storm.multilang.BoltMsg; +import backtype.storm.multilang.NoOutputException; +import backtype.storm.multilang.ShellMsg; +import backtype.storm.multilang.SpoutMsg; import backtype.storm.task.TopologyContext; -import java.io.BufferedReader; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.DataOutputStream; import java.io.File; import java.io.IOException; -import java.util.Map; +import java.io.InputStream; +import java.io.Serializable; import java.util.List; +import java.util.Map; import org.apache.commons.io.IOUtils; -import org.json.simple.JSONObject; -import org.json.simple.JSONValue; import org.apache.log4j.Logger; -public class ShellProcess { +public class ShellProcess implements Serializable { public static Logger LOG = Logger.getLogger(ShellProcess.class); - private DataOutputStream processIn; - private BufferedReader processOut; - private InputStream processErrorStream; - private Process _subprocess; - private String[] command; + public static Logger ShellLogger; + private Process _subprocess; + private InputStream processErrorStream; + private String[] command; + public ISerializer serializer; public ShellProcess(String[] command) { this.command = command; } - public Number launch(Map conf, TopologyContext context) throws IOException { + public Number launch(Map conf, TopologyContext context) { ProcessBuilder builder = new ProcessBuilder(command); builder.directory(new File(context.getCodeDir())); - _subprocess = builder.start(); - processIn = new DataOutputStream(_subprocess.getOutputStream()); - processOut = new BufferedReader(new InputStreamReader(_subprocess.getInputStream())); - processErrorStream = _subprocess.getErrorStream(); + ShellLogger = Logger.getLogger(context.getThisComponentId()); + + this.serializer = getSerializer(conf); + + Number pid; + try { + _subprocess = builder.start(); + processErrorStream = _subprocess.getErrorStream(); + serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream()); + pid = serializer.connect(conf, context); + } catch (IOException e) { + throw new RuntimeException( + "Error when launching multilang subprocess\n" + + getErrorsString(), e); + } catch (NoOutputException e) { + throw new RuntimeException(e + getErrorsString() + "\n"); + } + return pid; + } - JSONObject setupInfo = new JSONObject(); - setupInfo.put("pidDir", context.getPIDDir()); - setupInfo.put("conf", conf); - setupInfo.put("context", context); - writeMessage(setupInfo); + private ISerializer getSerializer(Map conf) { + //get factory class name + String serializer_className = (String)conf.get(Config.STORM_MULTILANG_SERIALIZER); + LOG.info("Storm multilang serializer:" + serializer_className); - return (Number)readMessage().get("pid"); + ISerializer serializer = null; + try { + //create a factory class + Class klass = Class.forName(serializer_className); + //obtain a serializer object + Object obj = klass.newInstance(); + serializer = (ISerializer)obj; + } catch(Exception e) { + throw new RuntimeException("Failed to construct multilang serializer from serializer " + serializer_className, e); + } + return serializer; } public void destroy() { _subprocess.destroy(); } - public void writeMessage(Object msg) throws IOException { - writeString(JSONValue.toJSONString(msg)); + public ShellMsg readShellMsg() throws IOException { + try { + return serializer.readShellMsg(); + } catch (NoOutputException e) { + throw new RuntimeException(e + getErrorsString() + "\n"); + } } - private void writeString(String str) throws IOException { - byte[] strBytes = str.getBytes("UTF-8"); - processIn.write(strBytes, 0, strBytes.length); - processIn.writeBytes("\nend\n"); - processIn.flush(); + public void writeBoltMsg(BoltMsg msg) throws IOException { + serializer.writeBoltMsg(msg); + // Log any info sent on the error stream + logErrorStream(); } - public JSONObject readMessage() throws IOException { - String string = readString(); - JSONObject msg = (JSONObject)JSONValue.parse(string); - if (msg != null) { - return msg; - } else { - throw new IOException("unable to parse: " + string); - } + public void writeSpoutMsg(SpoutMsg msg) throws IOException { + serializer.writeSpoutMsg(msg); + // Log any info sent on the error stream + logErrorStream(); } - public String getErrorsString() { - if(processErrorStream!=null) { - try { - return IOUtils.toString(processErrorStream); - } catch(IOException e) { - return "(Unable to capture error stream)"; - } - } else { - return ""; - } + public void writeTaskIds(List taskIds) throws IOException { + serializer.writeTaskIds(taskIds); + // Log any info sent on the error stream + logErrorStream(); } - public void drainErrorStream() - { + public void logErrorStream() { try { - while (processErrorStream.available() > 0) - { + while (processErrorStream.available() > 0) { int bufferSize = processErrorStream.available(); - byte[] errorReadingBuffer = new byte[bufferSize]; - + byte[] errorReadingBuffer = new byte[bufferSize]; processErrorStream.read(errorReadingBuffer, 0, bufferSize); - - LOG.info("Got error from shell process: " + new String(errorReadingBuffer)); + ShellLogger.info(new String(errorReadingBuffer)); } - } catch(Exception e) { + } catch (Exception e) { } } - private String readString() throws IOException { - StringBuilder line = new StringBuilder(); - - //synchronized (processOut) { - while (true) { - String subline = processOut.readLine(); - if(subline==null) { - StringBuilder errorMessage = new StringBuilder(); - errorMessage.append("Pipe to subprocess seems to be broken!"); - if (line.length() == 0) { - errorMessage.append(" No output read.\n"); - } - else { - errorMessage.append(" Currently read output: " + line.toString() + "\n"); - } - errorMessage.append("Shell Process Exception:\n"); - errorMessage.append(getErrorsString() + "\n"); - throw new RuntimeException(errorMessage.toString()); - } - if(subline.equals("end")) { - break; - } - if(line.length()!=0) { - line.append("\n"); - } - line.append(subline); + public String getErrorsString() { + if (processErrorStream != null) { + try { + return IOUtils.toString(processErrorStream); + } catch (IOException e) { + return "(Unable to capture error stream)"; } - //} - - return line.toString(); + } else { + return ""; + } } } diff --git a/storm-core/src/jvm/backtype/storm/utils/Time.java b/storm-core/src/jvm/backtype/storm/utils/Time.java index 506b0746e..50a79fd67 100644 --- a/storm-core/src/jvm/backtype/storm/utils/Time.java +++ b/storm-core/src/jvm/backtype/storm/utils/Time.java @@ -36,14 +36,18 @@ public class Time { private static AtomicLong simulatedCurrTimeMs; //should this be a thread local that's allowed to keep advancing? public static void startSimulating() { - simulating.set(true); - simulatedCurrTimeMs = new AtomicLong(0); - threadSleepTimes = new ConcurrentHashMap(); + synchronized(sleepTimesLock) { + simulating.set(true); + simulatedCurrTimeMs = new AtomicLong(0); + threadSleepTimes = new ConcurrentHashMap(); + } } public static void stopSimulating() { - simulating.set(false); - threadSleepTimes = null; + synchronized(sleepTimesLock) { + simulating.set(false); + threadSleepTimes = null; + } } public static boolean isSimulating() { @@ -61,7 +65,9 @@ public static void sleepUntil(long targetTimeMs) throws InterruptedException { } } finally { synchronized(sleepTimesLock) { - threadSleepTimes.remove(Thread.currentThread()); + if (simulating.get()) { + threadSleepTimes.remove(Thread.currentThread()); + } } } } else { diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java index 8a05fe495..c15e81717 100644 --- a/storm-core/src/jvm/backtype/storm/utils/Utils.java +++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java @@ -17,20 +17,12 @@ */ package backtype.storm.utils; -import backtype.storm.Config; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.ComponentObject; -import backtype.storm.generated.StormTopology; -import clojure.lang.IFn; -import clojure.lang.RT; -import com.netflix.curator.framework.CuratorFramework; -import com.netflix.curator.framework.CuratorFrameworkFactory; -import com.netflix.curator.retry.ExponentialBackoffRetry; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.FileOutputStream; -import java.io.InputStreamReader; import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.net.URL; @@ -46,11 +38,23 @@ import java.util.Map; import java.util.TreeMap; import java.util.UUID; + import org.apache.commons.lang.StringUtils; import org.apache.thrift.TException; import org.json.simple.JSONValue; import org.yaml.snakeyaml.Yaml; +import backtype.storm.Config; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.ComponentObject; +import backtype.storm.generated.StormTopology; +import clojure.lang.IFn; +import clojure.lang.RT; + +import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.CuratorFrameworkFactory; +import com.netflix.curator.retry.ExponentialBackoffRetry; + public class Utils { public static final String DEFAULT_STREAM_ID = "default"; @@ -135,7 +139,13 @@ public static Map findAndReadConfigFile(String name, boolean mustExist) { } URL resource = resources.iterator().next(); Yaml yaml = new Yaml(); - Map ret = (Map) yaml.load(new InputStreamReader(resource.openStream())); + Map ret = null; + InputStream input = resource.openStream(); + try { + ret = (Map) yaml.load(new InputStreamReader(input)); + } finally { + input.close(); + } if(ret==null) ret = new HashMap(); @@ -161,9 +171,13 @@ public static Map readCommandLineOpts() { commandOptions = commandOptions.replaceAll("%%%%", " "); String[] configs = commandOptions.split(","); for (String config : configs) { - String[] options = config.split("="); + String[] options = config.split("=", 2); if (options.length == 2) { - ret.put(options[0], options[1]); + Object val = JSONValue.parse(options[1]); + if (val == null) { + val = options[1]; + } + ret.put(options[0], val); } } } @@ -335,22 +349,18 @@ public static CuratorFramework newCurator(Map conf, List servers, Object serverPorts.add(zkServer + ":" + Utils.getInt(port)); } String zkStr = StringUtils.join(serverPorts, ",") + root; - try { - CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder() - .connectString(zkStr) - .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT))) - .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT))) - .retryPolicy(new BoundedExponentialBackoffRetry( - Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)), - Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)), - Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)))); - if(auth!=null && auth.scheme!=null) { - builder = builder.authorization(auth.scheme, auth.payload); - } - return builder.build(); - } catch (IOException e) { - throw new RuntimeException(e); + CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder() + .connectString(zkStr) + .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT))) + .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT))) + .retryPolicy(new BoundedExponentialBackoffRetry( + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)), + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)), + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)))); + if(auth!=null && auth.scheme!=null) { + builder = builder.authorization(auth.scheme, auth.payload); } + return builder.build(); } public static CuratorFramework newCurator(Map conf, List servers, Object port) { diff --git a/storm-core/src/jvm/storm/trident/state/map/OpaqueMap.java b/storm-core/src/jvm/storm/trident/state/map/OpaqueMap.java index cd6766d16..12f308364 100644 --- a/storm-core/src/jvm/storm/trident/state/map/OpaqueMap.java +++ b/storm-core/src/jvm/storm/trident/state/map/OpaqueMap.java @@ -43,7 +43,11 @@ public List multiGet(List> keys) { for(CachedBatchReadsMap.RetVal retval: curr) { OpaqueValue val = retval.val; if(val!=null) { - ret.add((T) val.get(_currTx)); + if(retval.cached) { + ret.add((T) val.getCurr()); + } else { + ret.add((T) val.get(_currTx)); + } } else { ret.add(null); } diff --git a/storm-core/src/jvm/storm/trident/state/map/RemovableMapState.java b/storm-core/src/jvm/storm/trident/state/map/RemovableMapState.java new file mode 100644 index 000000000..cf34f05e8 --- /dev/null +++ b/storm-core/src/jvm/storm/trident/state/map/RemovableMapState.java @@ -0,0 +1,8 @@ +package storm.trident.state.map; + +import java.util.List; +import storm.trident.state.State; + +public interface RemovableMapState extends State { + void multiRemove(List> keys); +} diff --git a/storm-core/src/jvm/storm/trident/testing/MemoryMapState.java b/storm-core/src/jvm/storm/trident/testing/MemoryMapState.java index 5df99f781..fd38900de 100644 --- a/storm-core/src/jvm/storm/trident/testing/MemoryMapState.java +++ b/storm-core/src/jvm/storm/trident/testing/MemoryMapState.java @@ -30,10 +30,13 @@ import storm.trident.state.map.*; import storm.trident.state.snapshot.Snapshottable; -public class MemoryMapState implements Snapshottable, ITupleCollection, MapState { +public class MemoryMapState implements Snapshottable, ITupleCollection, MapState, RemovableMapState { MemoryMapStateBacking _backing; SnapshottableMap _delegate; + List> _removed = new ArrayList(); + Long _currTx = null; + public MemoryMapState(String id) { _backing = new MemoryMapStateBacking(id); @@ -54,6 +57,11 @@ public T get() { public void beginCommit(Long txid) { _delegate.beginCommit(txid); + if(txid==null || !txid.equals(_currTx)) { + _backing.multiRemove(_removed); + } + _removed = new ArrayList(); + _currTx = txid; } public void commit(Long txid) { @@ -76,6 +84,17 @@ public List multiGet(List> keys) { return _delegate.multiGet(keys); } + @Override + public void multiRemove(List> keys) { + List nulls = new ArrayList(); + for(int i=0; i, T>) _dbs.get(id); } + public void multiRemove(List> keys) { + for(List key: keys) { + db.remove(key); + } + } + @Override public List multiGet(List> keys) { List ret = new ArrayList(); diff --git a/storm-core/test/clj/backtype/storm/config_test.clj b/storm-core/test/clj/backtype/storm/config_test.clj index 93c7df939..01f788ba7 100644 --- a/storm-core/test/clj/backtype/storm/config_test.clj +++ b/storm-core/test/clj/backtype/storm/config_test.clj @@ -83,3 +83,23 @@ (catch Exception e e)))) (is (thrown-cause? java.lang.IllegalArgumentException (.validateField validator "test" 42))))) + +(deftest test-worker-childopts-is-string-or-string-list + (let [pass-cases [nil "some string" ["some" "string" "list"]]] + (testing "worker.childopts validates" + (let [validator (CONFIG-SCHEMA-MAP WORKER-CHILDOPTS)] + (doseq [value pass-cases] + (is (nil? (try + (.validateField validator "test" value) + (catch Exception e e))))) + (is (thrown-cause? java.lang.IllegalArgumentException + (.validateField validator "test" 42))))) + + (testing "topology.worker.childopts validates" + (let [validator (CONFIG-SCHEMA-MAP TOPOLOGY-WORKER-CHILDOPTS)] + (doseq [value pass-cases] + (is (nil? (try + (.validateField validator "test" value) + (catch Exception e e))))) + (is (thrown-cause? java.lang.IllegalArgumentException + (.validateField validator "test" 42))))))) diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj index 9f9def98e..54a3ee9cd 100644 --- a/storm-core/test/clj/backtype/storm/supervisor_test.clj +++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj @@ -15,10 +15,12 @@ ;; limitations under the License. (ns backtype.storm.supervisor-test (:use [clojure test]) + (:require [clojure [string :as string]]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter]) - (:use [backtype.storm bootstrap testing]) + (:use [backtype.storm bootstrap config testing]) (:use [backtype.storm.daemon common]) (:require [backtype.storm.daemon [worker :as worker] [supervisor :as supervisor]]) + (:use [conjure core]) ) (bootstrap) @@ -239,6 +241,66 @@ ))) +(deftest test-worker-launch-command + (testing "*.worker.childopts configuration" + (let [mock-port "42" + mock-storm-id "fake-storm-id" + mock-worker-id "fake-worker-id" + mock-cp "mock-classpath" + exp-args-fn (fn [opts topo-opts] + (concat ["java" "-server"] + opts + topo-opts + ["-Djava.library.path=" + (str "-Dlogfile.name=worker-" mock-port ".log") + "-Dstorm.home=" + "-Dlogback.configurationFile=/logback/cluster.xml" + (str "-Dstorm.id=" mock-storm-id) + (str "-Dworker.id=" mock-worker-id) + (str "-Dworker.port=" mock-port) + "-cp" mock-cp + "backtype.storm.daemon.worker" + mock-storm-id + mock-port + mock-worker-id]))] + (testing "testing *.worker.childopts as strings with extra spaces" + (let [string-opts "-Dfoo=bar -Xmx1024m" + topo-string-opts "-Dkau=aux -Xmx2048m" + exp-args (exp-args-fn ["-Dfoo=bar" "-Xmx1024m"] + ["-Dkau=aux" "-Xmx2048m"]) + mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed + WORKER-CHILDOPTS string-opts}}] + (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS + topo-string-opts} + add-to-classpath mock-cp + supervisor-stormdist-root nil + launch-process nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id) + (verify-first-call-args-for-indices launch-process + [0] + exp-args)))) + (testing "testing *.worker.childopts as list of strings, with spaces in values" + (let [list-opts '("-Dopt1='this has a space in it'" "-Xmx1024m") + topo-list-opts '("-Dopt2='val with spaces'" "-Xmx2048m") + exp-args (exp-args-fn list-opts topo-list-opts) + mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed + WORKER-CHILDOPTS list-opts}}] + (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS + topo-list-opts} + add-to-classpath mock-cp + supervisor-stormdist-root nil + launch-process nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id) + (verify-first-call-args-for-indices launch-process + [0] + exp-args))))))) + (deftest test-workers-go-bananas ;; test that multiple workers are started for a port, and test that ;; supervisor shuts down propertly (doesn't shutdown the most diff --git a/storm-core/test/clj/backtype/storm/utils_test.clj b/storm-core/test/clj/backtype/storm/utils_test.clj index 99bf7d113..b24db623d 100644 --- a/storm-core/test/clj/backtype/storm/utils_test.clj +++ b/storm-core/test/clj/backtype/storm/utils_test.clj @@ -62,3 +62,14 @@ )) ) ) + +(deftest test-secs-to-millis-long + (is (= 0 (secs-to-millis-long 0))) + (is (= 2 (secs-to-millis-long 0.002))) + (is (= 500 (secs-to-millis-long 0.5))) + (is (= 1000 (secs-to-millis-long 1))) + (is (= 1080 (secs-to-millis-long 1.08))) + (is (= 10000 (secs-to-millis-long 10))) + (is (= 10100 (secs-to-millis-long 10.1))) +) + diff --git a/storm-core/test/clj/storm/trident/state_test.clj b/storm-core/test/clj/storm/trident/state_test.clj index 6e091e34f..63e38cab9 100644 --- a/storm-core/test/clj/storm/trident/state_test.clj +++ b/storm-core/test/clj/storm/trident/state_test.clj @@ -20,10 +20,16 @@ (:import [storm.trident.state OpaqueValue]) (:import [storm.trident.state CombinerValueUpdater]) (:import [storm.trident.state.map TransactionalMap OpaqueMap]) - (:import [storm.trident.testing MemoryBackingMap]) + (:import [storm.trident.testing MemoryBackingMap MemoryMapState]) (:use [storm.trident testing]) (:use [backtype.storm util])) +(defn single-remove [map key] + (-> map (.multiRemove [[key]]))) + +(defn single-put [map key val] + (-> map (.multiPut [[key]] [val]))) + (defn single-get [map key] (-> map (.multiGet [[key]]) first)) @@ -61,7 +67,9 @@ (is (= nil (single-get map "a"))) ;; tests that intra-batch caching works (is (= 1 (single-update map "a" 1))) + (is (= 1 (single-get map "a"))) (is (= 3 (single-update map "a" 2))) + (is (= 3 (single-get map "a"))) (.commit map 1) (.beginCommit map 1) (is (= nil (single-get map "a"))) @@ -94,3 +102,26 @@ (is (= 7 (single-update map "a" 1))) (.commit map 2) )) + + +(deftest test-memory-map-state-remove + (let [map (MemoryMapState. (uuid))] + (.beginCommit map 1) + (single-put map "a" 1) + (single-put map "b" 2) + (.commit map 1) + (.beginCommit map 2) + (single-remove map "a") + (is (nil? (single-get map "a"))) + (is (= 2 (single-get map "b"))) + (.commit map 2) + (.beginCommit map 2) + (is (= 1 (single-get map "a"))) + (is (= 2 (single-get map "b"))) + (single-remove map "a") + (.commit map 2) + (.beginCommit map 3) + (is (nil? (single-get map "a"))) + (is (= 2 (single-get map "b"))) + (.commit map 3) + )) diff --git a/storm-dist/binary/NOTICE b/storm-dist/binary/NOTICE index 0eec4766f..90ff43a15 100644 --- a/storm-dist/binary/NOTICE +++ b/storm-dist/binary/NOTICE @@ -4,13 +4,8 @@ Copyright 2013 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by Nathan Marz -Copyright 2011-2013 Nathan Marz - - This product includes software developed by Yahoo! Inc. (www.yahoo.com) -Copyright © 2012-2013 Yahoo! Inc. All rights reserved. +Copyright (c) 2012-2014 Yahoo! Inc. YAML support provided by snakeyaml (http://code.google.com/p/snakeyaml/). Copyright (c) 2008-2010 Andrey Somov @@ -33,4 +28,4 @@ Copyright (C) 2009 Fang Yidong and Chris Nokleberg Alternative collection types provided by google-collections from http://code.google.com/p/google-collections/. -Copyright (C) 2007 Google Inc. \ No newline at end of file +Copyright (C) 2007 Google Inc. diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 54ec172c0..31207094a 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.1-incubating-SNAPSHOT + 0.9.2-incubating-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/binary/src/main/assembly/binary.xml b/storm-dist/binary/src/main/assembly/binary.xml index 36a03190e..6e3670411 100644 --- a/storm-dist/binary/src/main/assembly/binary.xml +++ b/storm-dist/binary/src/main/assembly/binary.xml @@ -52,11 +52,42 @@ + + ${project.basedir}/../../examples + examples + + **/target/** + + + + + + ${project.basedir}/../../external/storm-kafka/target + external/storm-kafka + + storm*jar + + + + ${project.basedir}/../../external/storm-kafka + external/storm-kafka + + README.* + + + + + ${project.basedir}/../../examples/storm-starter/target/storm-starter-${project.version}-jar-with-dependencies.jar + /examples/storm-starter/ + storm-starter-topologies-${project.version}.jar + + + diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index d2a436628..83300946d 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.1-incubating-SNAPSHOT + 0.9.2-incubating-SNAPSHOT ../../pom.xml org.apache.storm