-
Notifications
You must be signed in to change notification settings - Fork 3.5k
/
PulsarSpout.java
481 lines (427 loc) · 19.9 KB
/
PulsarSpout.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
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES 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.pulsar.storm;
import static java.lang.String.format;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.Queue;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.pulsar.client.api.ClientBuilder;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Reader;
import org.apache.pulsar.client.impl.Backoff;
import org.apache.pulsar.client.impl.ClientBuilderImpl;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData;
import org.apache.pulsar.client.impl.conf.ReaderConfigurationData;
import org.apache.storm.metric.api.IMetric;
import org.apache.storm.spout.SpoutOutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.topology.base.BaseRichSpout;
import org.apache.storm.tuple.Values;
import org.apache.storm.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class PulsarSpout extends BaseRichSpout implements IMetric {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LoggerFactory.getLogger(PulsarSpout.class);
public static final String NO_OF_PENDING_FAILED_MESSAGES = "numberOfPendingFailedMessages";
public static final String NO_OF_MESSAGES_RECEIVED = "numberOfMessagesReceived";
public static final String NO_OF_MESSAGES_EMITTED = "numberOfMessagesEmitted";
public static final String NO_OF_MESSAGES_FAILED = "numberOfMessagesFailed";
public static final String MESSAGE_NOT_AVAILABLE_COUNT = "messageNotAvailableCount";
public static final String NO_OF_PENDING_ACKS = "numberOfPendingAcks";
public static final String CONSUMER_RATE = "consumerRate";
public static final String CONSUMER_THROUGHPUT_BYTES = "consumerThroughput";
private final ClientConfigurationData clientConf;
private final PulsarSpoutConfiguration pulsarSpoutConf;
private final ConsumerConfigurationData<byte[]> consumerConf;
private final long failedRetriesTimeoutNano;
private final int maxFailedRetries;
private final ConcurrentMap<MessageId, MessageRetries> pendingMessageRetries = new ConcurrentHashMap<>();
private final Queue<Message<byte[]>> failedMessages = new ConcurrentLinkedQueue<>();
private final ConcurrentMap<String, Object> metricsMap = new ConcurrentHashMap<>();
private SharedPulsarClient sharedPulsarClient;
private String componentId;
private String spoutId;
private SpoutOutputCollector collector;
private PulsarSpoutConsumer consumer;
private volatile long messagesReceived = 0;
private volatile long messagesEmitted = 0;
private volatile long messagesFailed = 0;
private volatile long messageNotAvailableCount = 0;
private volatile long pendingAcks = 0;
private volatile long messageSizeReceived = 0;
public PulsarSpout(PulsarSpoutConfiguration pulsarSpoutConf) {
this(pulsarSpoutConf, PulsarClient.builder());
}
public PulsarSpout(PulsarSpoutConfiguration pulsarSpoutConf, ClientBuilder clientBuilder) {
this(pulsarSpoutConf, ((ClientBuilderImpl) clientBuilder).getClientConfigurationData().clone(),
new ConsumerConfigurationData<byte[]>());
}
public PulsarSpout(PulsarSpoutConfiguration pulsarSpoutConf, ClientConfigurationData clientConfig,
ConsumerConfigurationData<byte[]> consumerConfig) {
Objects.requireNonNull(pulsarSpoutConf.getServiceUrl());
Objects.requireNonNull(pulsarSpoutConf.getTopic());
Objects.requireNonNull(pulsarSpoutConf.getSubscriptionName());
Objects.requireNonNull(pulsarSpoutConf.getMessageToValuesMapper());
checkNotNull(pulsarSpoutConf, "spout configuration can't be null");
checkNotNull(clientConfig, "client configuration can't be null");
checkNotNull(consumerConfig, "consumer configuration can't be null");
this.clientConf = clientConfig;
this.clientConf.setServiceUrl(pulsarSpoutConf.getServiceUrl());
this.consumerConf = consumerConfig;
this.pulsarSpoutConf = pulsarSpoutConf;
this.failedRetriesTimeoutNano = pulsarSpoutConf.getFailedRetriesTimeout(TimeUnit.NANOSECONDS);
this.maxFailedRetries = pulsarSpoutConf.getMaxFailedRetries();
}
@Override
public void close() {
try {
LOG.info("[{}] Closing Pulsar consumer for topic {}", spoutId, pulsarSpoutConf.getTopic());
if (pulsarSpoutConf.isAutoUnsubscribe()) {
try {
consumer.unsubscribe();
}catch(PulsarClientException e) {
LOG.error("[{}] Failed to unsubscribe {} on topic {}", spoutId,
this.pulsarSpoutConf.getSubscriptionName(), pulsarSpoutConf.getTopic(), e);
}
}
if (!pulsarSpoutConf.isSharedConsumerEnabled() && consumer != null) {
consumer.close();
}
if (sharedPulsarClient != null) {
sharedPulsarClient.close();
}
pendingMessageRetries.clear();
failedMessages.clear();
} catch (PulsarClientException e) {
LOG.error("[{}] Error closing Pulsar consumer for topic {}", spoutId, pulsarSpoutConf.getTopic(), e);
}
}
@Override
public void ack(Object msgId) {
if (msgId instanceof Message) {
Message<?> msg = (Message<?>) msgId;
if (LOG.isDebugEnabled()) {
LOG.debug("[{}] Received ack for message {}", spoutId, msg.getMessageId());
}
consumer.acknowledgeAsync(msg);
pendingMessageRetries.remove(msg.getMessageId());
// we should also remove message from failedMessages but it will be eventually removed while emitting next
// tuple
--pendingAcks;
}
}
@Override
public void fail(Object msgId) {
if (msgId instanceof Message) {
@SuppressWarnings("unchecked")
Message<byte[]> msg = (Message<byte[]>) msgId;
MessageId id = msg.getMessageId();
LOG.warn("[{}] Error processing message {}", spoutId, id);
// Since the message processing failed, we put it in the failed messages queue if there are more retries
// remaining for the message
MessageRetries messageRetries = pendingMessageRetries.computeIfAbsent(id, (k) -> new MessageRetries());
if ((failedRetriesTimeoutNano < 0
|| (messageRetries.getTimeStamp() + failedRetriesTimeoutNano) > System.nanoTime())
&& (maxFailedRetries < 0 || messageRetries.numRetries < maxFailedRetries)) {
// since we can retry again, we increment retry count and put it in the queue
LOG.info("[{}] Putting message {} in the retry queue", spoutId, id);
messageRetries.incrementAndGet();
pendingMessageRetries.putIfAbsent(id, messageRetries);
failedMessages.add(msg);
--pendingAcks;
messagesFailed++;
} else {
LOG.warn("[{}] Number of retries limit reached, dropping the message {}", spoutId, id);
ack(msg);
}
}
}
/**
* Emits a tuple received from the Pulsar consumer unless there are any failed messages
*/
@Override
public void nextTuple() {
emitNextAvailableTuple();
}
/**
* It makes sure that it emits next available non-tuple to topology unless consumer queue doesn't have any message
* available. It receives message from consumer queue and converts it to tuple and emits to topology. if the
* converted tuple is null then it tries to receives next message and perform the same until it finds non-tuple to
* emit.
*/
public void emitNextAvailableTuple() {
// check if there are any failed messages to re-emit in the topology
if(emitFailedMessage()) {
return;
}
Message<byte[]> msg;
// receive from consumer if no failed messages
if (consumer != null) {
if (LOG.isDebugEnabled()) {
LOG.debug("[{}] Receiving the next message from pulsar consumer to emit to the collector", spoutId);
}
try {
boolean done = false;
while (!done) {
msg = consumer.receive(100, TimeUnit.MILLISECONDS);
if (msg != null) {
++messagesReceived;
messageSizeReceived += msg.getData().length;
done = mapToValueAndEmit(msg);
} else {
// queue is empty and nothing to emit
done = true;
messageNotAvailableCount++;
}
}
} catch (PulsarClientException e) {
LOG.error("[{}] Error receiving message from pulsar consumer", spoutId, e);
}
}
}
private boolean emitFailedMessage() {
Message<byte[]> msg;
while ((msg = failedMessages.peek()) != null) {
MessageRetries messageRetries = pendingMessageRetries.get(msg.getMessageId());
if (messageRetries != null) {
// emit the tuple if retry doesn't need backoff else sleep with backoff time and return without doing
// anything
if (Backoff.shouldBackoff(messageRetries.getTimeStamp(), TimeUnit.NANOSECONDS,
messageRetries.getNumRetries(), clientConf.getInitialBackoffIntervalNanos(),
clientConf.getMaxBackoffIntervalNanos())) {
Utils.sleep(TimeUnit.NANOSECONDS.toMillis(clientConf.getInitialBackoffIntervalNanos()));
} else {
// remove the message from the queue and emit to the topology, only if it should not be backedoff
LOG.info("[{}] Retrying failed message {}", spoutId, msg.getMessageId());
failedMessages.remove();
mapToValueAndEmit(msg);
}
return true;
}
// messageRetries is null because messageRetries is already acked and removed from pendingMessageRetries
// then remove it from failed message queue as well.
if(LOG.isDebugEnabled()) {
LOG.debug("[{}]-{} removing {} from failedMessage because it's already acked",
pulsarSpoutConf.getTopic(), spoutId, msg.getMessageId());
}
failedMessages.remove();
// try to find out next failed message
continue;
}
return false;
}
@Override
@SuppressWarnings({ "rawtypes" })
public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
this.componentId = context.getThisComponentId();
this.spoutId = String.format("%s-%s", componentId, context.getThisTaskId());
this.collector = collector;
pendingMessageRetries.clear();
failedMessages.clear();
try {
consumer = createConsumer();
LOG.info("[{}] Created a pulsar consumer on topic {} to receive messages with subscription {}", spoutId,
pulsarSpoutConf.getTopic(), pulsarSpoutConf.getSubscriptionName());
} catch (PulsarClientException e) {
LOG.error("[{}] Error creating pulsar consumer on topic {}", spoutId, pulsarSpoutConf.getTopic(), e);
throw new IllegalStateException(format("Failed to initialize consumer for %s-%s : %s",
pulsarSpoutConf.getTopic(), pulsarSpoutConf.getSubscriptionName(), e.getMessage()), e);
}
context.registerMetric(String.format("PulsarSpoutMetrics-%s-%s", componentId, context.getThisTaskIndex()), this,
pulsarSpoutConf.getMetricsTimeIntervalInSecs());
}
private PulsarSpoutConsumer createConsumer() throws PulsarClientException {
sharedPulsarClient = SharedPulsarClient.get(componentId, clientConf);
PulsarSpoutConsumer consumer;
if (pulsarSpoutConf.isSharedConsumerEnabled()) {
consumer = pulsarSpoutConf.isDurableSubscription()
? new SpoutConsumer(sharedPulsarClient.getSharedConsumer(newConsumerConfiguration()))
: new SpoutReader(sharedPulsarClient.getSharedReader(newReaderConfiguration()));
} else {
try {
consumer = pulsarSpoutConf.isDurableSubscription()
? new SpoutConsumer(sharedPulsarClient.getClient()
.subscribeAsync(newConsumerConfiguration()).join())
: new SpoutReader(sharedPulsarClient.getClient()
.createReaderAsync(newReaderConfiguration()).join());
} catch (CompletionException e) {
throw (PulsarClientException) e.getCause();
}
}
return consumer;
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
pulsarSpoutConf.getMessageToValuesMapper().declareOutputFields(declarer);
}
private boolean mapToValueAndEmit(Message<byte[]> msg) {
if (msg != null) {
Values values = pulsarSpoutConf.getMessageToValuesMapper().toValues(msg);
++pendingAcks;
if (values == null) {
// since the mapper returned null, we can drop the message and ack it immediately
if (LOG.isDebugEnabled()) {
LOG.debug("[{}] Dropping message {}", spoutId, msg.getMessageId());
}
ack(msg);
} else {
collector.emit(values, msg);
++messagesEmitted;
if (LOG.isDebugEnabled()) {
LOG.debug("[{}] Emitted message {} to the collector", spoutId, msg.getMessageId());
}
return true;
}
}
return false;
}
public class MessageRetries {
private final long timestampInNano;
private int numRetries;
public MessageRetries() {
this.timestampInNano = System.nanoTime();
this.numRetries = 0;
}
public long getTimeStamp() {
return timestampInNano;
}
public int incrementAndGet() {
return ++numRetries;
}
public int getNumRetries() {
return numRetries;
}
}
/**
* Helpers for metrics
*/
@SuppressWarnings({ "rawtypes" })
ConcurrentMap getMetrics() {
metricsMap.put(NO_OF_PENDING_FAILED_MESSAGES, (long) pendingMessageRetries.size());
metricsMap.put(NO_OF_MESSAGES_RECEIVED, messagesReceived);
metricsMap.put(NO_OF_MESSAGES_EMITTED, messagesEmitted);
metricsMap.put(NO_OF_MESSAGES_FAILED, messagesFailed);
metricsMap.put(MESSAGE_NOT_AVAILABLE_COUNT, messageNotAvailableCount);
metricsMap.put(NO_OF_PENDING_ACKS, pendingAcks);
metricsMap.put(CONSUMER_RATE, ((double) messagesReceived) / pulsarSpoutConf.getMetricsTimeIntervalInSecs());
metricsMap.put(CONSUMER_THROUGHPUT_BYTES,
((double) messageSizeReceived) / pulsarSpoutConf.getMetricsTimeIntervalInSecs());
return metricsMap;
}
void resetMetrics() {
messagesReceived = 0;
messagesEmitted = 0;
messageSizeReceived = 0;
messagesFailed = 0;
messageNotAvailableCount = 0;
}
@SuppressWarnings("rawtypes")
@Override
public Object getValueAndReset() {
ConcurrentMap metrics = getMetrics();
resetMetrics();
return metrics;
}
private ReaderConfigurationData<byte[]> newReaderConfiguration() {
ReaderConfigurationData<byte[]> readerConf = new ReaderConfigurationData<>();
readerConf.setTopicName(pulsarSpoutConf.getTopic());
readerConf.setReaderName(pulsarSpoutConf.getSubscriptionName());
readerConf.setStartMessageId(pulsarSpoutConf.getNonDurableSubscriptionReadPosition());
if (this.consumerConf != null) {
readerConf.setCryptoFailureAction(consumerConf.getCryptoFailureAction());
readerConf.setCryptoKeyReader(consumerConf.getCryptoKeyReader());
readerConf.setReadCompacted(consumerConf.isReadCompacted());
readerConf.setReceiverQueueSize(consumerConf.getReceiverQueueSize());
}
return readerConf;
}
private ConsumerConfigurationData<byte[]> newConsumerConfiguration() {
ConsumerConfigurationData<byte[]> consumerConf = this.consumerConf != null ? this.consumerConf
: new ConsumerConfigurationData<>();
consumerConf.setTopicNames(Collections.singleton(pulsarSpoutConf.getTopic()));
consumerConf.setSubscriptionName(pulsarSpoutConf.getSubscriptionName());
consumerConf.setSubscriptionType(pulsarSpoutConf.getSubscriptionType());
return consumerConf;
}
static class SpoutConsumer implements PulsarSpoutConsumer {
private Consumer<byte[]> consumer;
public SpoutConsumer(Consumer<byte[]> consumer) {
super();
this.consumer = consumer;
}
@Override
public Message<byte[]> receive(int timeout, TimeUnit unit) throws PulsarClientException {
return consumer.receive(timeout, unit);
}
@Override
public void acknowledgeAsync(Message<?> msg) {
consumer.acknowledgeAsync(msg);
}
@Override
public void close() throws PulsarClientException {
consumer.close();
}
@Override
public void unsubscribe() throws PulsarClientException {
consumer.unsubscribe();
}
}
static class SpoutReader implements PulsarSpoutConsumer {
private Reader<byte[]> reader;
public SpoutReader(Reader<byte[]> reader) {
super();
this.reader = reader;
}
@Override
public Message<byte[]> receive(int timeout, TimeUnit unit) throws PulsarClientException {
return reader.readNext(timeout, unit);
}
@Override
public void acknowledgeAsync(Message<?> msg) {
// No-op
}
@Override
public void close() throws PulsarClientException {
try {
reader.close();
} catch (IOException e) {
throw new PulsarClientException(e);
}
}
@Override
public void unsubscribe() throws PulsarClientException {
// No-op
}
}
}