├── .gitignore ├── AUTHORS ├── .settings ├── org.eclipse.m2e.core.prefs └── org.eclipse.jdt.core.prefs ├── .classpath ├── .project ├── LICENSE.MIT ├── pom.xml ├── src └── main │ └── java │ └── com │ └── rapportive │ └── storm │ ├── amqp │ ├── QueueDeclaration.java │ ├── ExclusiveQueueWithBinding.java │ └── SharedQueueWithBinding.java │ └── spout │ └── AMQPSpout.java └── README.markdown /.gitignore: -------------------------------------------------------------------------------- 1 | target 2 | -------------------------------------------------------------------------------- /AUTHORS: -------------------------------------------------------------------------------- 1 | Sam Stokes 2 | Olga Gorun 3 | -------------------------------------------------------------------------------- /.settings/org.eclipse.m2e.core.prefs: -------------------------------------------------------------------------------- 1 | #Sat Sep 24 23:46:12 PDT 2011 2 | activeProfiles= 3 | eclipse.preferences.version=1 4 | resolveWorkspaceProjects=true 5 | version=1 6 | -------------------------------------------------------------------------------- /.classpath: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | -------------------------------------------------------------------------------- /.settings/org.eclipse.jdt.core.prefs: -------------------------------------------------------------------------------- 1 | #Sat Sep 24 23:46:38 PDT 2011 2 | eclipse.preferences.version=1 3 | org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled 4 | org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6 5 | org.eclipse.jdt.core.compiler.compliance=1.6 6 | org.eclipse.jdt.core.compiler.problem.assertIdentifier=error 7 | org.eclipse.jdt.core.compiler.problem.enumIdentifier=error 8 | org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning 9 | org.eclipse.jdt.core.compiler.source=1.6 10 | -------------------------------------------------------------------------------- /.project: -------------------------------------------------------------------------------- 1 | 2 | 3 | storm-amqp-spout 4 | 5 | 6 | 7 | 8 | 9 | org.eclipse.jdt.core.javabuilder 10 | 11 | 12 | 13 | 14 | org.eclipse.m2e.core.maven2Builder 15 | 16 | 17 | 18 | 19 | 20 | org.eclipse.jdt.core.javanature 21 | org.eclipse.m2e.core.maven2Nature 22 | 23 | 24 | -------------------------------------------------------------------------------- /LICENSE.MIT: -------------------------------------------------------------------------------- 1 | Copyright (c) 2011 Sam Stokes 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy 4 | of this software and associated documentation files (the "Software"), to deal 5 | in the Software without restriction, including without limitation the rights 6 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | copies of the Software, and to permit persons to whom the Software is 8 | furnished to do so, subject to the following conditions: 9 | 10 | The above copyright notice and this permission notice shall be included in 11 | all copies or substantial portions of the Software. 12 | 13 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | THE SOFTWARE. 20 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4.0.0 3 | com.rapportive 4 | storm-amqp-spout 5 | 0.1.1 6 | 7 | 8 | clojars.org 9 | http://clojars.org/repo 10 | 11 | 12 | 13 | 14 | storm 15 | storm 16 | 0.5.4 17 | 18 | 19 | com.rabbitmq 20 | amqp-client 21 | 2.6.1 22 | 23 | 24 | log4j 25 | log4j 26 | 1.2.16 27 | 28 | 29 | 30 | 31 | 32 | org.apache.maven.plugins 33 | maven-javadoc-plugin 34 | 2.8 35 | 36 | 37 | http://nathanmarz.github.com/storm/doc 38 | 39 | 40 | 41 | 42 | 43 | 44 | -------------------------------------------------------------------------------- /src/main/java/com/rapportive/storm/amqp/QueueDeclaration.java: -------------------------------------------------------------------------------- 1 | package com.rapportive.storm.amqp; 2 | 3 | import java.io.IOException; 4 | import java.io.Serializable; 5 | 6 | import com.rabbitmq.client.AMQP.Queue; 7 | 8 | import com.rabbitmq.client.Channel; 9 | 10 | /** 11 | * Declaration of a queue to consume, and any exchange bindings the queue needs. 12 | * 13 | *

Depending on the queue parameters (exclusive, auto_delete, server-named) 14 | * and exchange bindings, it may or may not be safe to start several consumers 15 | * in parallel using a given queue declaration. For example, an exclusive 16 | * named queue bound to an exchange is not safe because only one of the 17 | * consumers will succeed in declaring the queue; an exclusive 18 | * server-named queue does not have that problem, but is still 19 | * probably not safe, because most exchange types will send a copy of every 20 | * message to every queue bound to them, so you will end up consuming each 21 | * message several times.

22 | * 23 | *

For that reason, to implement this interface you must implement 24 | * {@link #isParallelConsumable} to indicate whether or not this queue is safe 25 | * for parallel consumers.

26 | */ 27 | public interface QueueDeclaration extends Serializable { 28 | /** 29 | * Declare the queue, and any exchanges and bindings that it needs. Called 30 | * once to determine the queue to consume from. 31 | * 32 | * @param channel An open AMQP channel which can be used to send the 33 | * declarations. 34 | * 35 | * @return the server's response to the successful queue declaration (used 36 | * to determine the queue name to subscribe to). 37 | * 38 | * @throws IOException if a declaration fails or the AMQP connection drops. 39 | */ 40 | Queue.DeclareOk declare(Channel channel) throws IOException; 41 | 42 | /** 43 | * Indicate whether this queue is safe for parallel consumers. 44 | * 45 | * @return true if safe for parallel consumers, otherwise 46 | * false. 47 | */ 48 | boolean isParallelConsumable(); 49 | } 50 | -------------------------------------------------------------------------------- /src/main/java/com/rapportive/storm/amqp/ExclusiveQueueWithBinding.java: -------------------------------------------------------------------------------- 1 | package com.rapportive.storm.amqp; 2 | 3 | import java.io.IOException; 4 | 5 | import com.rabbitmq.client.AMQP.Queue; 6 | 7 | import com.rabbitmq.client.Channel; 8 | 9 | /** 10 | * Declares an exclusive, server-named queue and binds it to an existing 11 | * exchange. This is probably the easiest way to start prototyping with an 12 | * {@link com.rapportive.storm.spout.AMQPSpout}: if your app already publishes 13 | * to an exchange, you can just point this at the exchange and start consuming 14 | * messages. 15 | * 16 | *

However N.B. this queue setup is not reliable, 17 | * in that if the spout task crashes or restarts, messages published while the 18 | * spout is down will be lost (because the spout creates the queue when it 19 | * starts up, and the server deletes the queue when the spout closes).

20 | * 21 | *

It also cannot scale out to multiple parallel spout tasks. The semantics 22 | * of an exclusive queue mean that each spout task would get its own queue 23 | * bound to the exchange. That means each task would receive a copy of every 24 | * message, so messages would get processed multiple times.

25 | * 26 | *

If you need guaranteed processing or a horizontally scalable spout, 27 | * consider {@link SharedQueueWithBinding}.

28 | */ 29 | public class ExclusiveQueueWithBinding implements QueueDeclaration { 30 | private static final long serialVersionUID = 7923072289071634425L; 31 | 32 | private final String exchange; 33 | private final String routingKey; 34 | 35 | /** 36 | * Create a declaration of an exclusive server-named queue bound to the 37 | * specified exchange. 38 | * 39 | * @param exchange exchange to bind the queue to. 40 | * @param routingKey routing key for the exchange binding. Use "#" to 41 | * receive all messages published to the exchange. 42 | */ 43 | public ExclusiveQueueWithBinding(String exchange, String routingKey) { 44 | this.exchange = exchange; 45 | this.routingKey = routingKey; 46 | } 47 | 48 | /** 49 | * Verifies the exchange exists, creates an exclusive, server-named queue 50 | * and binds it to the exchange. 51 | * 52 | * @return the server's response to the successful queue declaration (you 53 | * can use this to discover the name of the queue). 54 | * 55 | * @throws IOException if the exchange does not exist, or if the AMQP 56 | * connection drops. 57 | */ 58 | @Override 59 | public Queue.DeclareOk declare(Channel channel) throws IOException { 60 | channel.exchangeDeclarePassive(exchange); 61 | 62 | final Queue.DeclareOk queue = channel.queueDeclare(); 63 | 64 | channel.queueBind(queue.getQueue(), exchange, routingKey); 65 | 66 | return queue; 67 | } 68 | 69 | /** 70 | * Returns false as this queue is not safe for parallel 71 | * consumers. 72 | */ 73 | @Override 74 | public boolean isParallelConsumable() { 75 | return false; 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/java/com/rapportive/storm/amqp/SharedQueueWithBinding.java: -------------------------------------------------------------------------------- 1 | package com.rapportive.storm.amqp; 2 | 3 | import java.io.IOException; 4 | 5 | import com.rabbitmq.client.AMQP.Queue; 6 | 7 | import com.rabbitmq.client.Channel; 8 | 9 | /** 10 | * Declares a named, durable queue and binds it to an existing exchange. This 11 | * is a good choice for production use as the queue will survive spout 12 | * restarts, so you won't miss messages if your spout crashes. 13 | * 14 | *

N.B. this could be risky under some circumstances. e.g. 15 | * if while prototyping you set a development topology consuming from a 16 | * production AMQP server, then kill your topology and go home for the night; 17 | * messages will continue to be queued up, which could threaten the stability 18 | * of the AMQP server if the exchange is high-volume. For prototyping consider 19 | * {@link ExclusiveQueueWithBinding}.

20 | * 21 | *

This queue is safe for multiple parallel spout tasks: as they all consume 22 | * the same named queue, the AMQP broker will round-robin messages between 23 | * them, so each message will get processed only once (barring redelivery due 24 | * to outages).

25 | */ 26 | public class SharedQueueWithBinding implements QueueDeclaration { 27 | private static final long serialVersionUID = 2364833412534518859L; 28 | 29 | private final String queueName; 30 | private final String exchange; 31 | private final String routingKey; 32 | 33 | /** 34 | * Create a declaration of a named, durable, non-exclusive queue bound to 35 | * the specified exchange. 36 | * 37 | * @param queueName name of the queue to be declared. 38 | * @param exchange exchange to bind the queue to. 39 | * @param routingKey routing key for the exchange binding. Use "#" to 40 | * receive all messages published to the exchange. 41 | */ 42 | public SharedQueueWithBinding(String queueName, String exchange, String routingKey) { 43 | this.queueName = queueName; 44 | this.exchange = exchange; 45 | this.routingKey = routingKey; 46 | } 47 | 48 | /** 49 | * Verifies the exchange exists, creates the named queue if it does not 50 | * exist, and binds it to the exchange. 51 | * 52 | * @return the server's response to the successful queue declaration. 53 | * 54 | * @throws IOException if the exchange does not exist, the queue could not 55 | * be declared, or if the AMQP connection drops. 56 | */ 57 | @Override 58 | public Queue.DeclareOk declare(Channel channel) throws IOException { 59 | channel.exchangeDeclarePassive(exchange); 60 | 61 | final Queue.DeclareOk queue = channel.queueDeclare( 62 | queueName, 63 | /* durable */ true, 64 | /* non-exclusive */ false, 65 | /* non-auto-delete */ false, 66 | /* no arguments */ null); 67 | 68 | channel.queueBind(queue.getQueue(), exchange, routingKey); 69 | 70 | return queue; 71 | } 72 | 73 | /** 74 | * Returns true as this queue is safe for parallel consumers. 75 | */ 76 | @Override 77 | public boolean isParallelConsumable() { 78 | return true; 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /README.markdown: -------------------------------------------------------------------------------- 1 | # storm-amqp-spout: AMQP input source for Storm # 2 | 3 | ## This is out of date! 4 | 5 | This is a rather old version of the spout that is not API-compatible with newer 6 | versions of Storm. Various people have forked it to bring it up to date and add 7 | features: 8 | 9 | * https://github.com/Xorlev/storm-amqp-spout 10 | * https://github.com/dkincaid/storm-amqp-spout 11 | 12 | If you want your fork added to this list, or to contribute a description of 13 | your fork, please email sam@rapportive.com. 14 | 15 | ## Description 16 | 17 | storm-amqp-spout allows a [Storm][] topology to consume an AMQP queue as an 18 | input source. It currently provides: 19 | 20 | * [AMQPSpout][]: an implementation of 21 | [`backtype.storm.topology.IRichSpout`][IRichSpout] that connects to an AMQP 22 | broker, consumes the messages routed to a specified AMQP queue and emits them 23 | as Storm tuples. 24 | * [QueueDeclaration][]: an interface that encapsulates declaring an 25 | AMQP queue and setting up any exchange bindings it requires, used by 26 | AMQPSpout to set up the queue to consume. 27 | * [ExclusiveQueueWithBinding][]: a QueueDeclaration suitable for 28 | prototyping and one-off analytics use cases. 29 | * [SharedQueueWithBinding][]: a QueueDeclaration suitable for 30 | production use cases needing guaranteed processing. 31 | 32 | You'll need to provide a [Scheme][] to tell AMQPSpout how to interpret the 33 | messages and turn them into Storm tuples. See e.g. [storm-json][] if your 34 | messages are JSON. 35 | 36 | ## Documentation ## 37 | 38 | The Javadocs can be found at [http://code.rapportive.com/storm-amqp-spout](). 39 | 40 | ## Usage ## 41 | 42 | To produce a jar: 43 | 44 | $ mvn package 45 | 46 | To install in your local Maven repository: 47 | 48 | $ mvn install 49 | 50 | To use in your `pom.xml`: 51 | 52 | ```xml 53 | 54 | 55 | 56 | 57 | 58 | com.rapportive 59 | storm-amqp-spout 60 | 0.1.1 61 | 62 | 63 | 64 | 65 | 66 | ``` 67 | 68 | ## Caveats ## 69 | 70 | This is early software. It has been used at production volumes, but not yet 71 | for business-critical use cases. It may break and the API is liable to change 72 | completely between releases. Pull requests, patches and bug reports are very 73 | welcome. 74 | 75 | **N.B.** if you need to guarantee all messages are reliably processed, you 76 | should have AMQPSpout consume from a queue that is *not* set as 'exclusive' or 77 | 'auto-delete': otherwise if the spout task crashes or is restarted, the queue 78 | will be deleted and any messages in it lost, as will any messages published 79 | while the task remains down. See [SharedQueueWithBinding][] to declare a 80 | shared queue that allows for guaranteed processing. (For prototyping, an 81 | [ExclusiveQueueWithBinding][] may be simpler to manage.) 82 | 83 | This does not currently handle malformed messages very well: the spout worker 84 | will crash if the provided [Scheme][] fails to deserialise a message. 85 | 86 | This does not currently support retrying messages in the event of transient 87 | failure to process: any message which the topology fails to process will simply 88 | be dropped. This is to prevent infinite redelivery in the event of 89 | non-transient failures (e.g. malformed messages, though see previous caveat!). 90 | This will probably be made configurable in a future release. 91 | 92 | ## Compatibility ## 93 | 94 | AMQPSpout has been tested with RabbitMQ 2.3.1, 2.6.1 and 2.7.0. It should probably work with other 95 | versions and other AMQP brokers. 96 | 97 | 98 | [Storm]: 99 | "Storm project homepage" 100 | [IRichSpout]: 101 | "Javadoc for backtype.storm.topology.IRichSpout" 102 | [Scheme]: 103 | "Javadoc for backtype.storm.spout.Scheme" 104 | [AMQPSpout]: 105 | "Javadoc for AMQPSpout" 106 | [QueueDeclaration]: 107 | "Javadoc for QueueDeclaration" 108 | [ExclusiveQueueWithBinding]: 109 | "Javadoc for ExclusiveQueueWithBinding" 110 | [SharedQueueWithBinding]: 111 | "Javadoc for SharedQueueWithBinding" 112 | [storm-json]: 113 | "JSON {,de}serialisation support for Storm" 114 | -------------------------------------------------------------------------------- /src/main/java/com/rapportive/storm/spout/AMQPSpout.java: -------------------------------------------------------------------------------- 1 | package com.rapportive.storm.spout; 2 | 3 | import java.io.IOException; 4 | import java.util.Map; 5 | 6 | import org.apache.log4j.Logger; 7 | 8 | import com.rabbitmq.client.AMQP.Queue; 9 | 10 | import com.rabbitmq.client.Channel; 11 | import com.rabbitmq.client.Connection; 12 | import com.rabbitmq.client.ConnectionFactory; 13 | import com.rabbitmq.client.QueueingConsumer; 14 | import com.rabbitmq.client.ShutdownSignalException; 15 | 16 | import com.rapportive.storm.amqp.QueueDeclaration; 17 | import backtype.storm.spout.Scheme; 18 | import backtype.storm.spout.SpoutOutputCollector; 19 | import backtype.storm.task.TopologyContext; 20 | import backtype.storm.topology.IRichSpout; 21 | import backtype.storm.topology.OutputFieldsDeclarer; 22 | 23 | import backtype.storm.utils.Utils; 24 | 25 | /** 26 | * Spout to feed messages into Storm from an AMQP queue. Each message routed 27 | * to the queue will be emitted as a Storm tuple. The message will be acked or 28 | * rejected once the topology has respectively fully processed or failed the 29 | * corresponding tuple. 30 | * 31 | *

N.B. if you need to guarantee all messages are reliably 32 | * processed, you should have AMQPSpout consume from a queue that is 33 | * not set as 'exclusive' or 'auto-delete': otherwise if the spout 34 | * task crashes or is restarted, the queue will be deleted and any messages in 35 | * it lost, as will any messages published while the task remains down. See 36 | * {@link com.rapportive.storm.amqp.SharedQueueWithBinding} to declare a shared 37 | * queue that allows for guaranteed processing. (For prototyping, an 38 | * {@link com.rapportive.storm.amqp.ExclusiveQueueWithBinding} may be 39 | * simpler to manage.)

40 | * 41 | *

N.B. this does not currently handle malformed messages 42 | * (which cannot be deserialised by the provided {@link Scheme}) very well: 43 | * the spout worker will crash if it fails to serialise a message.

44 | * 45 | *

This consumes messages from AMQP asynchronously, so it may receive 46 | * messages before Storm requests them as tuples; therefore it buffers messages 47 | * in an internal queue. To avoid this buffer growing large and consuming too 48 | * much RAM, set {@link #CONFIG_PREFETCH_COUNT}.

49 | * 50 | *

This spout can be distributed among multiple workers, depending on the 51 | * queue declaration: see {@link QueueDeclaration#isParallelConsumable}.

52 | * 53 | * @see QueueDeclaration 54 | * @see com.rapportive.storm.amqp.SharedQueueWithBinding 55 | * @see com.rapportive.storm.amqp.ExclusiveQueueWithBinding 56 | * 57 | * @author Sam Stokes (sam@rapportive.com) 58 | */ 59 | public class AMQPSpout implements IRichSpout { 60 | private static final long serialVersionUID = 11258942292629263L; 61 | 62 | private static final Logger log = Logger.getLogger(AMQPSpout.class); 63 | 64 | /** 65 | * Storm config key to set the AMQP basic.qos prefetch-count parameter. 66 | * Defaults to 100. 67 | * 68 | *

This caps the number of messages outstanding (i.e. unacked) at a time 69 | * that will be sent to each spout worker. Increasing this will improve 70 | * throughput if the network roundtrip time to the AMQP broker is 71 | * significant compared to the time for the topology to process each 72 | * message; this will also increase the RAM requirements as the internal 73 | * message buffer grows.

74 | * 75 | *

AMQP allows a prefetch-count of zero, indicating unlimited delivery, 76 | * but that is not allowed here to avoid unbounded buffer growth.

77 | */ 78 | public static final String CONFIG_PREFETCH_COUNT = "amqp.prefetch.count"; 79 | private static final long DEFAULT_PREFETCH_COUNT = 100; 80 | 81 | /** 82 | * Time in milliseconds to wait for a message from the queue if there is 83 | * no message ready when the topology requests a tuple (via 84 | * {@link #nextTuple()}). 85 | */ 86 | public static final long WAIT_FOR_NEXT_MESSAGE = 1L; 87 | 88 | /** 89 | * Time in milliseconds to wait after losing connection to the AMQP broker 90 | * before attempting to reconnect. 91 | */ 92 | public static final long WAIT_AFTER_SHUTDOWN_SIGNAL = 10000L; 93 | 94 | private final String amqpHost; 95 | private final int amqpPort; 96 | private final String amqpUsername; 97 | private final String amqpPassword; 98 | private final String amqpVhost; 99 | 100 | private final QueueDeclaration queueDeclaration; 101 | 102 | private final Scheme serialisationScheme; 103 | 104 | private transient Connection amqpConnection; 105 | private transient Channel amqpChannel; 106 | private transient QueueingConsumer amqpConsumer; 107 | private transient String amqpConsumerTag; 108 | 109 | private SpoutOutputCollector collector; 110 | 111 | private int prefetchCount; 112 | 113 | 114 | /** 115 | * Create a new AMQP spout. When 116 | * {@link #open(Map, TopologyContext, SpoutOutputCollector)} is called, it 117 | * will declare a queue according to the specified 118 | * queueDeclaration, subscribe to the queue, and start consuming 119 | * messages. It will use the provided scheme to deserialise each 120 | * AMQP message into a Storm tuple. 121 | * 122 | * @param host hostname of the AMQP broker node 123 | * @param port port number of the AMQP broker node 124 | * @param username username to log into to the broker 125 | * @param password password to authenticate to the broker 126 | * @param vhost vhost on the broker 127 | * @param queueDeclaration declaration of the queue / exchange bindings 128 | * @param scheme {@link backtype.storm.spout.Scheme} used to deserialise 129 | * each AMQP message into a Storm tuple 130 | */ 131 | public AMQPSpout(String host, int port, String username, String password, String vhost, QueueDeclaration queueDeclaration, Scheme scheme) { 132 | this.amqpHost = host; 133 | this.amqpPort = port; 134 | this.amqpUsername = username; 135 | this.amqpPassword = password; 136 | this.amqpVhost = vhost; 137 | this.queueDeclaration = queueDeclaration; 138 | 139 | this.serialisationScheme = scheme; 140 | } 141 | 142 | 143 | /** 144 | * Acks the message with the AMQP broker. 145 | */ 146 | @Override 147 | public void ack(Object msgId) { 148 | if (msgId instanceof Long) { 149 | final long deliveryTag = (Long) msgId; 150 | if (amqpChannel != null) { 151 | try { 152 | amqpChannel.basicAck(deliveryTag, false /* not multiple */); 153 | } catch (IOException e) { 154 | log.warn("Failed to ack delivery-tag " + deliveryTag, e); 155 | } 156 | } 157 | } else { 158 | log.warn(String.format("don't know how to ack(%s: %s)", msgId.getClass().getName(), msgId)); 159 | } 160 | } 161 | 162 | 163 | /** 164 | * Cancels the queue subscription, and disconnects from the AMQP broker. 165 | */ 166 | @Override 167 | public void close() { 168 | try { 169 | if (amqpChannel != null) { 170 | if (amqpConsumerTag != null) { 171 | amqpChannel.basicCancel(amqpConsumerTag); 172 | } 173 | 174 | amqpChannel.close(); 175 | } 176 | } catch (IOException e) { 177 | log.warn("Error closing AMQP channel", e); 178 | } 179 | 180 | try { 181 | if (amqpConnection != null) { 182 | amqpConnection.close(); 183 | } 184 | } catch (IOException e) { 185 | log.warn("Error closing AMQP connection", e); 186 | } 187 | } 188 | 189 | 190 | /** 191 | * Tells the AMQP broker to drop (Basic.Reject) the message. 192 | * 193 | *

N.B. this does not requeue the message: 194 | * failed messages will simply be dropped. This is to prevent infinite 195 | * redelivery in the event of non-transient failures (e.g. malformed 196 | * messages). However it means that messages will not be retried 197 | * in the event of transient failures.

198 | * 199 | *

TODO make this configurable.

200 | */ 201 | @Override 202 | public void fail(Object msgId) { 203 | if (msgId instanceof Long) { 204 | final long deliveryTag = (Long) msgId; 205 | if (amqpChannel != null) { 206 | try { 207 | amqpChannel.basicReject(deliveryTag, false /* don't requeue */); 208 | } catch (IOException e) { 209 | log.warn("Failed to reject delivery-tag " + deliveryTag, e); 210 | } 211 | } 212 | } else { 213 | log.warn(String.format("don't know how to reject(%s: %s)", msgId.getClass().getName(), msgId)); 214 | } 215 | } 216 | 217 | 218 | /** 219 | * Emits the next message from the queue as a tuple. 220 | * 221 | *

If no message is ready to emit, this will wait a short time 222 | * ({@link #WAIT_FOR_NEXT_MESSAGE}) for one to arrive on the queue, 223 | * to avoid a tight loop in the spout worker.

224 | */ 225 | @Override 226 | public void nextTuple() { 227 | if (amqpConsumer != null) { 228 | try { 229 | final QueueingConsumer.Delivery delivery = amqpConsumer.nextDelivery(WAIT_FOR_NEXT_MESSAGE); 230 | if (delivery == null) return; 231 | final long deliveryTag = delivery.getEnvelope().getDeliveryTag(); 232 | final byte[] message = delivery.getBody(); 233 | collector.emit(serialisationScheme.deserialize(message), deliveryTag); 234 | /* 235 | * TODO what to do about malformed messages? Skip? 236 | * Avoid infinite retry! 237 | * Maybe we should output them on a separate stream. 238 | */ 239 | } catch (ShutdownSignalException e) { 240 | log.warn("AMQP connection dropped, will attempt to reconnect..."); 241 | Utils.sleep(WAIT_AFTER_SHUTDOWN_SIGNAL); 242 | reconnect(); 243 | } catch (InterruptedException e) { 244 | // interrupted while waiting for message, big deal 245 | } 246 | } 247 | } 248 | 249 | 250 | /** 251 | * Connects to the AMQP broker, declares the queue and subscribes to 252 | * incoming messages. 253 | */ 254 | @Override 255 | public void open(@SuppressWarnings("rawtypes") Map config, TopologyContext context, SpoutOutputCollector collector) { 256 | Long prefetchCount = (Long) config.get(CONFIG_PREFETCH_COUNT); 257 | if (prefetchCount == null) { 258 | log.info("Using default prefetch-count"); 259 | prefetchCount = DEFAULT_PREFETCH_COUNT; 260 | } else if (prefetchCount < 1) { 261 | throw new IllegalArgumentException(CONFIG_PREFETCH_COUNT + " must be at least 1"); 262 | } 263 | this.prefetchCount = prefetchCount.intValue(); 264 | 265 | try { 266 | this.collector = collector; 267 | 268 | setupAMQP(); 269 | } catch (IOException e) { 270 | log.error("AMQP setup failed", e); 271 | } 272 | } 273 | 274 | 275 | private void setupAMQP() throws IOException { 276 | final int prefetchCount = this.prefetchCount; 277 | 278 | final ConnectionFactory connectionFactory = new ConnectionFactory(); 279 | 280 | connectionFactory.setHost(amqpHost); 281 | connectionFactory.setPort(amqpPort); 282 | connectionFactory.setUsername(amqpUsername); 283 | connectionFactory.setPassword(amqpPassword); 284 | connectionFactory.setVirtualHost(amqpVhost); 285 | 286 | this.amqpConnection = connectionFactory.newConnection(); 287 | this.amqpChannel = amqpConnection.createChannel(); 288 | 289 | log.info("Setting basic.qos prefetch-count to " + prefetchCount); 290 | amqpChannel.basicQos(prefetchCount); 291 | 292 | final Queue.DeclareOk queue = queueDeclaration.declare(amqpChannel); 293 | final String queueName = queue.getQueue(); 294 | log.info("Consuming queue " + queueName); 295 | 296 | this.amqpConsumer = new QueueingConsumer(amqpChannel); 297 | this.amqpConsumerTag = amqpChannel.basicConsume(queueName, false /* no auto-ack */, amqpConsumer); 298 | } 299 | 300 | 301 | private void reconnect() { 302 | log.info("Reconnecting to AMQP broker..."); 303 | try { 304 | setupAMQP(); 305 | } catch (IOException e) { 306 | log.warn("Failed to reconnect to AMQP broker", e); 307 | } 308 | } 309 | 310 | 311 | /** 312 | * Declares the output fields of this spout according to the provided 313 | * {@link backtype.storm.spout.Scheme}. 314 | */ 315 | @Override 316 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 317 | declarer.declare(serialisationScheme.getOutputFields()); 318 | } 319 | 320 | 321 | /** 322 | * This spout can be distributed among multiple workers if the 323 | * {@link QueueDeclaration} supports it. 324 | * 325 | * @see QueueDeclaration#isParallelConsumable() 326 | */ 327 | @Override 328 | public boolean isDistributed() { 329 | return queueDeclaration.isParallelConsumable(); 330 | } 331 | } 332 | --------------------------------------------------------------------------------