/
JmsSpout.java
382 lines (333 loc) · 13 KB
/
JmsSpout.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.storm.jms.spout;
import java.io.Serializable;
import java.util.*;
import java.util.concurrent.LinkedBlockingQueue;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.Destination;
import javax.jms.JMSException;
import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.MessageListener;
import javax.jms.Session;
import org.apache.storm.topology.base.BaseRichSpout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.storm.jms.JmsProvider;
import org.apache.storm.jms.JmsTupleProducer;
import org.apache.storm.spout.SpoutOutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.tuple.Values;
import org.apache.storm.utils.Utils;
/**
* A Storm <code>Spout</code> implementation that listens to a JMS topic or queue
* and outputs tuples based on the messages it receives.
* <p>
* <code>JmsSpout</code> instances rely on <code>JmsProducer</code> implementations
* to obtain the JMS <code>ConnectionFactory</code> and <code>Destination</code> objects
* necessary to connect to a JMS topic/queue.
* <p>
* When a <code>JmsSpout</code> receives a JMS message, it delegates to an
* internal <code>JmsTupleProducer</code> instance to create a Storm tuple from the
* incoming message.
* <p>
* Typically, developers will supply a custom <code>JmsTupleProducer</code> implementation
* appropriate for the expected message content.
*/
@SuppressWarnings("serial")
public class JmsSpout extends BaseRichSpout implements MessageListener {
private static final Logger LOG = LoggerFactory.getLogger(JmsSpout.class);
// JMS options
private int jmsAcknowledgeMode = Session.AUTO_ACKNOWLEDGE;
private boolean distributed = true;
private JmsTupleProducer tupleProducer;
private JmsProvider jmsProvider;
private LinkedBlockingQueue<Message> queue;
private TreeSet<JmsMessageID> toCommit;
private HashMap<JmsMessageID, Message> pendingMessages;
private long messageSequence = 0;
private SpoutOutputCollector collector;
private transient Connection connection;
private transient Session session;
private boolean hasFailures = false;
public final Serializable recoveryMutex = "RECOVERY_MUTEX";
private Timer recoveryTimer = null;
private long recoveryPeriod = -1; // default to disabled
/**
* Sets the JMS Session acknowledgement mode for the JMS seesion associated with this spout.
* <p>
* Possible values:
* <ul>
* <li>javax.jms.Session.AUTO_ACKNOWLEDGE</li>
* <li>javax.jms.Session.CLIENT_ACKNOWLEDGE</li>
* <li>javax.jms.Session.DUPS_OK_ACKNOWLEDGE</li>
* </ul>
*
* @param mode JMS Session Acknowledgement mode
* @throws IllegalArgumentException if the mode is not recognized.
*/
public void setJmsAcknowledgeMode(int mode) {
switch (mode) {
case Session.AUTO_ACKNOWLEDGE:
case Session.CLIENT_ACKNOWLEDGE:
case Session.DUPS_OK_ACKNOWLEDGE:
break;
default:
throw new IllegalArgumentException("Unknown Acknowledge mode: " + mode + " (See javax.jms.Session for valid values)");
}
this.jmsAcknowledgeMode = mode;
}
/**
* Returns the JMS Session acknowledgement mode for the JMS seesion associated with this spout.
*
* @return
*/
public int getJmsAcknowledgeMode() {
return this.jmsAcknowledgeMode;
}
/**
* Set the <code>JmsProvider</code>
* implementation that this Spout will use to connect to
* a JMS <code>javax.jms.Desination</code>
*
* @param provider
*/
public void setJmsProvider(JmsProvider provider) {
this.jmsProvider = provider;
}
/**
* Set the <code>JmsTupleProducer</code>
* implementation that will convert <code>javax.jms.Message</code>
* object to <code>org.apache.storm.tuple.Values</code> objects
* to be emitted.
*
* @param producer
*/
public void setJmsTupleProducer(JmsTupleProducer producer) {
this.tupleProducer = producer;
}
/**
* <code>javax.jms.MessageListener</code> implementation.
* <p>
* Stored the JMS message in an internal queue for processing
* by the <code>nextTuple()</code> method.
*/
public void onMessage(Message msg) {
try {
LOG.debug("Queuing msg [" + msg.getJMSMessageID() + "]");
} catch (JMSException e) {
}
this.queue.offer(msg);
}
/**
* <code>ISpout</code> implementation.
* <p>
* Connects the JMS spout to the configured JMS destination
* topic/queue.
*/
@SuppressWarnings("rawtypes")
public void open(Map conf, TopologyContext context,
SpoutOutputCollector collector) {
if (this.jmsProvider == null) {
throw new IllegalStateException("JMS provider has not been set.");
}
if (this.tupleProducer == null) {
throw new IllegalStateException("JMS Tuple Producer has not been set.");
}
Integer topologyTimeout = (Integer) conf.get("topology.message.timeout.secs");
// TODO fine a way to get the default timeout from storm, so we're not hard-coding to 30 seconds (it could change)
topologyTimeout = topologyTimeout == null ? 30 : topologyTimeout;
if ((topologyTimeout.intValue() * 1000) > this.recoveryPeriod) {
LOG.warn("*** WARNING *** : " +
"Recovery period (" + this.recoveryPeriod + " ms.) is less then the configured " +
"'topology.message.timeout.secs' of " + topologyTimeout +
" secs. This could lead to a message replay flood!");
}
this.queue = new LinkedBlockingQueue<Message>();
this.toCommit = new TreeSet<JmsMessageID>();
this.pendingMessages = new HashMap<JmsMessageID, Message>();
this.collector = collector;
try {
ConnectionFactory cf = this.jmsProvider.connectionFactory();
Destination dest = this.jmsProvider.destination();
this.connection = cf.createConnection();
this.session = connection.createSession(false,
this.jmsAcknowledgeMode);
MessageConsumer consumer = session.createConsumer(dest);
consumer.setMessageListener(this);
this.connection.start();
if (this.isDurableSubscription() && this.recoveryPeriod > 0) {
this.recoveryTimer = new Timer();
this.recoveryTimer.scheduleAtFixedRate(new RecoveryTask(), 10, this.recoveryPeriod);
}
} catch (Exception e) {
LOG.warn("Error creating JMS connection.", e);
}
}
public void close() {
try {
LOG.debug("Closing JMS connection.");
this.session.close();
this.connection.close();
} catch (JMSException e) {
LOG.warn("Error closing JMS connection.", e);
}
}
public void nextTuple() {
Message msg = this.queue.poll();
if (msg == null) {
Utils.sleep(50);
} else {
LOG.debug("sending tuple: " + msg);
// get the tuple from the handler
try {
Values vals = this.tupleProducer.toTuple(msg);
// ack if we're not in AUTO_ACKNOWLEDGE mode, or the message requests ACKNOWLEDGE
LOG.debug("Requested deliveryMode: " + toDeliveryModeString(msg.getJMSDeliveryMode()));
LOG.debug("Our deliveryMode: " + toDeliveryModeString(this.jmsAcknowledgeMode));
if (this.isDurableSubscription()) {
LOG.debug("Requesting acks.");
JmsMessageID messageId = new JmsMessageID(this.messageSequence++, msg.getJMSMessageID());
this.collector.emit(vals, messageId);
// at this point we successfully emitted. Store
// the message and message ID so we can do a
// JMS acknowledge later
this.pendingMessages.put(messageId, msg);
this.toCommit.add(messageId);
} else {
this.collector.emit(vals);
}
} catch (JMSException e) {
LOG.warn("Unable to convert JMS message: " + msg);
}
}
}
/*
* Will only be called if we're transactional or not AUTO_ACKNOWLEDGE
*/
public void ack(Object msgId) {
Message msg = this.pendingMessages.remove(msgId);
JmsMessageID oldest = this.toCommit.first();
if (msgId.equals(oldest)) {
if (msg != null) {
try {
LOG.debug("Committing...");
msg.acknowledge();
LOG.debug("JMS Message acked: " + msgId);
this.toCommit.remove(msgId);
} catch (JMSException e) {
LOG.warn("Error acknowldging JMS message: " + msgId, e);
}
} else {
LOG.warn("Couldn't acknowledge unknown JMS message ID: " + msgId);
}
} else {
this.toCommit.remove(msgId);
}
}
/*
* Will only be called if we're transactional or not AUTO_ACKNOWLEDGE
*/
public void fail(Object msgId) {
LOG.warn("Message failed: " + msgId);
this.pendingMessages.clear();
this.toCommit.clear();
synchronized (this.recoveryMutex) {
this.hasFailures = true;
}
}
public void declareOutputFields(OutputFieldsDeclarer declarer) {
this.tupleProducer.declareOutputFields(declarer);
}
/**
* Returns <code>true</code> if the spout has received failures
* from which it has not yet recovered.
*/
public boolean hasFailures() {
return this.hasFailures;
}
protected void recovered() {
this.hasFailures = false;
}
/**
* Sets the periodicity of the timer task that
* checks for failures and recovers the JMS session.
*
* @param period
*/
public void setRecoveryPeriod(long period) {
this.recoveryPeriod = period;
}
public boolean isDistributed() {
return this.distributed;
}
/**
* Sets the "distributed" mode of this spout.
* <p>
* If <code>true</code> multiple instances of this spout <i>may</i> be
* created across the cluster (depending on the "parallelism_hint" in the topology configuration).
* <p>
* Setting this value to <code>false</code> essentially means this spout will run as a singleton
* within the cluster ("parallelism_hint" will be ignored).
* <p>
* In general, this should be set to <code>false</code> if the underlying JMS destination is a
* topic, and <code>true</code> if it is a JMS queue.
*
* @param distributed
*/
public void setDistributed(boolean distributed) {
this.distributed = distributed;
}
private static final String toDeliveryModeString(int deliveryMode) {
switch (deliveryMode) {
case Session.AUTO_ACKNOWLEDGE:
return "AUTO_ACKNOWLEDGE";
case Session.CLIENT_ACKNOWLEDGE:
return "CLIENT_ACKNOWLEDGE";
case Session.DUPS_OK_ACKNOWLEDGE:
return "DUPS_OK_ACKNOWLEDGE";
default:
return "UNKNOWN";
}
}
protected Session getSession() {
return this.session;
}
private boolean isDurableSubscription() {
return (this.jmsAcknowledgeMode != Session.AUTO_ACKNOWLEDGE);
}
private class RecoveryTask extends TimerTask {
private final Logger LOG = LoggerFactory.getLogger(RecoveryTask.class);
public void run() {
synchronized (JmsSpout.this.recoveryMutex) {
if (JmsSpout.this.hasFailures()) {
try {
LOG.info("Recovering from a message failure.");
JmsSpout.this.getSession().recover();
JmsSpout.this.recovered();
} catch (JMSException e) {
LOG.warn("Could not recover jms session.", e);
}
}
}
}
}
}