This repository has been archived by the owner on Feb 12, 2022. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 13
/
Consumer.java
670 lines (579 loc) · 27.4 KB
/
Consumer.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
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
/**
* Copyright (c) 2017, Salesforce.com, Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without modification, are permitted provided that the
* following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice, this list of conditions and the following
* disclaimer.
*
* * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following
* disclaimer in the documentation and/or other materials provided with the distribution.
*
* * Neither the name of Salesforce.com nor the names of its contributors may be used to endorse or promote products
* derived from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES,
* INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
* DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
* SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
* WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE
* USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
package com.salesforce.storm.spout.dynamic.kafka;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.salesforce.storm.spout.dynamic.ConsumerPartition;
import com.salesforce.storm.spout.dynamic.FactoryManager;
import com.salesforce.storm.spout.dynamic.consumer.ConsumerPeerContext;
import com.salesforce.storm.spout.dynamic.consumer.PartitionDistributor;
import com.salesforce.storm.spout.dynamic.VirtualSpoutIdentifier;
import com.salesforce.storm.spout.dynamic.consumer.ConsumerState;
import com.salesforce.storm.spout.dynamic.consumer.PartitionOffsetsManager;
import com.salesforce.storm.spout.dynamic.consumer.Record;
import com.salesforce.storm.spout.dynamic.kafka.deserializer.Deserializer;
import com.salesforce.storm.spout.dynamic.persistence.PersistenceAdapter;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.InvalidOffsetException;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.internals.Topic;
import org.apache.storm.tuple.Values;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Clock;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* A high level kafka consumer that handles state/log offset management in a way that supports
* marking messages as being processed in non-sequential order. On consumer restarts, this implementation
* errs on the side of re-playing a previously processed message (at least once semantics) over accidentally skipping
* un-processed messages. This means there exists certain scenarios where it could replay previously processed
* messages.
*
* How does this Consumer track completed offsets?
* This consumer will emit messages out the same sequential order as it consumes it from Kafka. However there
* is no guarantee what order those messages will get processed by a storm topology. The topology could process
* and ack those messages in any order. So lets imagine the following scenario:
* Emit Offsets: 0,1,2,3,4,5
*
* For whatever reason offset #3 takes longer to process, so we get acks in the following order back from storm:
* Ack Offsets: 0,1,4,5,2
*
* At this point internally this consumer knows it has processed the above offsets, but is missing offset #3.
* This consumer tracks completed offsets sequentially, meaning it will mark offset #2 as being the last finished offset
* because it is the largest offset that we know we have acked every offset preceding it. If at this point the topology
* was stopped, and the consumer shut down, when the topology was redeployed, this consumer would resume consuming at
* offset #3, re-emitting the following:
* Emit Offsets: 3,4,5
*
* Now imagine the following acks come in:
* Ack Offsets: 4,5,3
*
* Internally the consumer will recognize that offsets 3 through 5 are all complete, and now mark offset #5 as the last
* finished offset.
*/
// TODO - rename this class?
public class Consumer implements com.salesforce.storm.spout.dynamic.consumer.Consumer {
// For logging.
private static final Logger logger = LoggerFactory.getLogger(Consumer.class);
// Kafka Consumer Instance and its Config.
private KafkaConsumerConfig consumerConfig;
private KafkaConsumer<byte[], byte[]> kafkaConsumer;
/**
* Boolean to prevent double initialization.
*/
private boolean isOpen = false;
/**
* State/offset management.
* ConsumerStateManager - Used to manage persisting consumer state, and when the consumer is restarted,
* loading the last known consumer state back in.
*/
private PersistenceAdapter persistenceAdapter;
/**
* Our Deserializer, it deserializes messages from Kafka into objects.
*/
private Deserializer deserializer;
/**
* Since offsets are managed on a per partition basis, each namespace/partition has its own ConsumerPartitionStateManagers
* instance to track its own offset. The state of these are what gets persisted via the ConsumerStateManager.
*/
private final PartitionOffsetsManager partitionOffsetsManager = new PartitionOffsetsManager();
/**
* Used to buffers messages read from Kafka.
*/
private ConsumerRecords<byte[], byte[]> buffer = null;
private Iterator<ConsumerRecord<byte[], byte[]>> bufferIterator = null;
/**
* Default constructor.
*/
public Consumer() {
}
/**
* Constructor used for testing, allows for injecting a KafkaConsumer instance.
*/
Consumer(final KafkaConsumer<byte[], byte[]> kafkaConsumer) {
this();
this.kafkaConsumer = kafkaConsumer;
}
/**
* Get the kafka consumer, if it has been retried yet, set it up.
* @return Kafka consumer
*/
KafkaConsumer<byte[], byte[]> getKafkaConsumer() {
// If kafkaConsumer is not null, we'll create one.
// If it is NOT null, we'll re-use the instance we got passed in from the constructor.
// Typically you'd pass in an instance for testing.
if (kafkaConsumer == null) {
// Construct new consumer
kafkaConsumer = new KafkaConsumer<>(getConsumerConfig().getKafkaConsumerProperties());
}
return kafkaConsumer;
}
/**
* Handles connecting to the Kafka cluster, determining which partitions to subscribe to,
* and based on previously saved state from ConsumerStateManager, seek to the last positions processed on
* each partition.
* @param spoutConfig Configuration of Spout.
* @param virtualSpoutIdentifier VirtualSpout running this consumer.
* @param consumerPeerContext defines how many instances in total are running of this consumer.
* @param persistenceAdapter The persistence adapter used to manage any state.
* @param startingState (Optional) If not null, This defines the state at which the consumer should resume from.
*/
public void open(
final Map<String, Object> spoutConfig,
final VirtualSpoutIdentifier virtualSpoutIdentifier,
final ConsumerPeerContext consumerPeerContext,
final PersistenceAdapter persistenceAdapter,
final ConsumerState startingState
) {
// Simple state enforcement.
if (isOpen) {
throw new IllegalStateException("Cannot call open more than once.");
}
isOpen = true;
// Build KafkaConsumerConfig from spoutConfig
final List<String> kafkaBrokers = (List<String>) spoutConfig.get(KafkaConsumerConfig.KAFKA_BROKERS);
final String topic = (String) spoutConfig.get(KafkaConsumerConfig.KAFKA_TOPIC);
Preconditions.checkArgument(
!kafkaBrokers.isEmpty(),
"Kafka brokers are required"
);
Preconditions.checkArgument(
topic != null && !topic.isEmpty(),
"Kafka topic is required"
);
// TODO ConsumerConfig should use a VirtualSpoutIdentifier
final KafkaConsumerConfig consumerConfig = new KafkaConsumerConfig(kafkaBrokers, virtualSpoutIdentifier.toString(), topic);
// Use ConsumerPeerContext to setup how many instances we have.
consumerConfig.setNumberOfConsumers(
consumerPeerContext.getTotalInstances()
);
consumerConfig.setIndexOfConsumer(
consumerPeerContext.getInstanceNumber()
);
// Create deserializer.
final Deserializer deserializer = FactoryManager.createNewInstance(
(String) spoutConfig.get(KafkaConsumerConfig.DESERIALIZER_CLASS)
);
// Save references
this.consumerConfig = consumerConfig;
this.persistenceAdapter = persistenceAdapter;
this.deserializer = deserializer;
// Get partitions
List<TopicPartition> topicPartitions = getPartitions();
if (topicPartitions.isEmpty()) {
throw new RuntimeException("Cannot assign partitions when there are none!");
}
logger.info("Assigning namespace and partitions = {}", topicPartitions);
// Assign our consumer to the given partitions
getKafkaConsumer().assign(topicPartitions);
for (TopicPartition topicPartition : topicPartitions) {
Long startingOffset = null;
if (startingState != null) {
startingOffset = startingState.getOffsetForNamespaceAndPartition(topicPartition.topic(), topicPartition.partition());
}
// Check to see if we have an existing offset saved for this partition
Long offset = persistenceAdapter.retrieveConsumerState(getConsumerId(), topicPartition.partition());
if (offset == null && startingOffset != null) {
offset = startingOffset;
}
// If we have a non-null offset
if (offset != null) {
// We have a stored offset, so pick up on the partition where we left off
logger.info(
"Resuming namespace {} partition {} at offset {}",
topicPartition.topic(),
topicPartition.partition(),
(offset + 1)
);
getKafkaConsumer().seek(topicPartition, (offset + 1));
} else {
// We do not have an existing offset saved, so start from the head
getKafkaConsumer().seekToBeginning(Collections.singletonList(topicPartition));
// This preserve the 0.10.0.x behavior where a seekToBeginning() call followed by position() on an
// otherwise empty partition would yield us a -1. In 0.11.0.x it throws this exception if the
// partition is empty.
try {
offset = getKafkaConsumer().position(topicPartition) - 1;
} catch (InvalidOffsetException ex) {
logger.info("{} appears to be empty!", topicPartition);
offset = -1L;
}
logger.info(
"Starting at the beginning of namespace {} partition {} => offset {}",
topicPartition.topic(), topicPartition.partition(), offset
);
}
// Start tracking offsets on ConsumerPartition
partitionOffsetsManager.replaceEntry(
new ConsumerPartition(topicPartition.topic(), topicPartition.partition()),
offset
);
}
}
/**
* Ask the consumer for the next message from Kafka.
* @return The next Record read from kafka, or null if no such msg is available.
*/
public Record nextRecord() {
// Fill our buffer if its empty
fillBuffer();
// Check our iterator for the next message
if (!bufferIterator.hasNext()) {
// Oh no! No new msg found.
logger.debug("Unable to fill buffer...nothing new!");
return null;
}
// Iterate to next result
final ConsumerRecord<byte[], byte[]> nextRecord = bufferIterator.next();
// Create consumerPartition instance
final ConsumerPartition consumerPartition = new ConsumerPartition(nextRecord.topic(), nextRecord.partition());
// Track this new message's state
partitionOffsetsManager.startOffset(consumerPartition, nextRecord.offset());
// Deserialize into values
final Values deserializedValues = getDeserializer().deserialize(
nextRecord.topic(),
nextRecord.partition(),
nextRecord.offset(),
nextRecord.key(),
nextRecord.value()
);
// Handle null
if (deserializedValues == null) {
// Failed to deserialize, just ack and return null?
logger.debug("Deserialization returned null");
// Mark as completed.
commitOffset(consumerPartition, nextRecord.offset());
// return null
return null;
}
// Return the record
return new Record(nextRecord.topic(), nextRecord.partition(), nextRecord.offset(), deserializedValues);
}
/**
* Mark a particular offset on a Topic/Partition as having been successfully processed.
* @param consumerPartition The Topic & Partition the offset belongs to
* @param offset The offset that should be marked as completed.
*/
private void commitOffset(final ConsumerPartition consumerPartition, final long offset) {
// Track internally which offsets we've marked completed
partitionOffsetsManager.finishOffset(consumerPartition, offset);
}
/**
* Marks a particular offset on a Topic/Partition as having been successfully processed.
* @param namespace The topic offset belongs to.
* @param partition The partition the offset belongs to.
* @param offset The offset that should be marked as completed.
*/
public void commitOffset(final String namespace, final int partition, final long offset) {
commitOffset(new ConsumerPartition(namespace, partition), offset);
}
/**
* Forces the Consumer's current state to be persisted.
* @return A copy of the state that was persisted.
*/
public ConsumerState flushConsumerState() {
// Get the current state
final ConsumerState consumerState = partitionOffsetsManager.getCurrentState();
// Persist each partition offset
for (Map.Entry<ConsumerPartition, Long> entry: consumerState.entrySet()) {
final ConsumerPartition consumerPartition = entry.getKey();
final long lastFinishedOffset = entry.getValue();
// Persist it.
persistenceAdapter.persistConsumerState(
getConsumerId(),
consumerPartition.partition(),
lastFinishedOffset
);
}
// return the state.
return consumerState;
}
/**
* Internal method used to fill internal message buffer from kafka.
* Maybe this should be marked private.
*/
private void fillBuffer() {
// If our buffer is null, or our iterator is at the end
if (buffer == null || !bufferIterator.hasNext()) {
// If we have no assigned partitions to consume from, then don't call poll()
// The underlying consumer call here does NOT make an API call, so this is safe to call within this loop.
if (getKafkaConsumer().assignment().isEmpty()) {
// No assigned partitions, nothing to consume :)
return;
}
// Time to refill the buffer
try {
buffer = getKafkaConsumer().poll(300);
} catch (OffsetOutOfRangeException outOfRangeException) {
// Handle it
handleOffsetOutOfRange(outOfRangeException);
// Clear out so we can attempt next time.
buffer = null;
bufferIterator = null;
// TODO: heh... this should be bounded most likely...
fillBuffer();
return;
}
// Create new iterator
bufferIterator = buffer.iterator();
}
}
/**
* This method handles when a partition seek/retrieve request was out of bounds.
* This happens in two scenarios:
* 1 - The offset is too old and was cleaned up / removed by the broker.
* 2 - The offset just plain does not exist.
*
* This is particularly nasty in that if the poll() was able to pull SOME messages from
* SOME partitions before the exception was thrown, those messages are considered "consumed"
* by KafkaClient, and there's no way to get them w/o seeking back to them for those partitions.
*
* This means when we roll back, we may replay some messages :/
*
* @param outOfRangeException The exception that was raised by the consumer.
*/
private void handleOffsetOutOfRange(final OffsetOutOfRangeException outOfRangeException) {
final Set<TopicPartition> resetPartitions = Sets.newHashSet();
// Loop over all the partitions in this exception
for (final TopicPartition topicPartition : outOfRangeException.offsetOutOfRangePartitions().keySet()) {
// The offset that was in the error
final long exceptionOffset = outOfRangeException.offsetOutOfRangePartitions().get(topicPartition);
// What kafka says the last offset is
final long endingOffset = getKafkaConsumer().endOffsets(Collections.singletonList(topicPartition))
.get(topicPartition);
logger.warn("Offset Out of Range for partition {} at offset {}, kafka says last offset in partition is {}",
topicPartition.partition(), exceptionOffset, endingOffset);
// We have a hypothesis that the consumer can actually seek past the last message of the topic,
// this yields this error and we want to catch it and try to back it up just a bit to a place that
// we can work from.
if (exceptionOffset >= endingOffset) {
logger.warn(
"OutOfRangeException yielded offset {}, which is past our ending offset of {} for {}",
exceptionOffset,
endingOffset,
topicPartition
);
// Seek to the end we found above. The end may have moved since we last asked, which is why we are not doing seekToEnd()
getKafkaConsumer().seek(
topicPartition,
endingOffset
);
partitionOffsetsManager.replaceEntry(
new ConsumerPartition(topicPartition.topic(), topicPartition.partition()),
endingOffset
);
} else {
resetPartitions.add(topicPartition);
}
}
// All of the error'd partitions we need to seek to earliest available position.
resetPartitionsToEarliest(resetPartitions);
}
/**
* Internal method that given a collection of topic partitions will find the earliest
* offset for that partition, seek the underlying consumer to it, and reset its internal
* offset tracking to that new position.
*
* This should be used when no state exists for a given partition, OR if the offset
* requested was too old.
* @param topicPartitions The collection of offsets to reset offsets for to the earliest position.
*/
private void resetPartitionsToEarliest(Collection<TopicPartition> topicPartitions) {
if (topicPartitions.isEmpty()) {
logger.info("Reset partitions requested with no partitions supplied.");
return;
}
// Seek to earliest for each
logger.info("Seeking to earliest offset on partitions {}", topicPartitions);
// If you call this with an empty set it resets everything that the consumer is assigned, which is probably
// not what you want...
getKafkaConsumer().seekToBeginning(topicPartitions);
// Now for each partition
for (TopicPartition topicPartition: topicPartitions) {
// Determine the current offset now that we've seeked to earliest
// We subtract one from this offset and set that as the last "committed" offset.
final long newOffset = getKafkaConsumer().position(topicPartition) - 1;
// We need to reset the saved offset to the current value
// Replace PartitionOffsetManager with new instance from new position.
logger.info("Partition {} using new earliest offset {}", topicPartition, newOffset);
partitionOffsetsManager.replaceEntry(
new ConsumerPartition(topicPartition.topic(), topicPartition.partition()), newOffset
);
}
}
/**
* Close out Kafka connections.
*/
public void close() {
// If our consumer is already null
if (kafkaConsumer == null) {
// Do nothing.
return;
}
// Close out persistence manager.
persistenceAdapter.close();
// Call close on underlying consumer
kafkaConsumer.close();
kafkaConsumer = null;
}
/**
* @return The defined consumer config.
*/
KafkaConsumerConfig getConsumerConfig() {
return consumerConfig;
}
/**
* @return PersistenceAdapter instance.
*/
public PersistenceAdapter getPersistenceAdapter() {
return persistenceAdapter;
}
/**
* @return A set of all the partitions currently subscribed to.
*/
Set<ConsumerPartition> getAssignedPartitions() {
// Create our return set using abstracted TopicPartition
Set<ConsumerPartition> assignedPartitions = new HashSet<>();
// Loop over resumes from underlying kafka consumer
for (TopicPartition topicPartition: getKafkaConsumer().assignment()) {
// Convert object type
assignedPartitions.add(new ConsumerPartition(topicPartition.topic(), topicPartition.partition()));
}
// Return immutable copy of our list.
return Collections.unmodifiableSet(assignedPartitions);
}
/**
* Unsubscribe the consumer from a specific topic/partition.
* @param consumerPartitionToUnsubscribe the Topic/Partition to stop consuming from.
* @return boolean, true if unsubscribed, false if it did not.
*/
public boolean unsubscribeConsumerPartition(final ConsumerPartition consumerPartitionToUnsubscribe) {
// Determine what we're currently assigned to,
// We clone the returned set so we can modify it.
final Set<ConsumerPartition> assignedTopicPartitions = Sets.newHashSet(getAssignedPartitions());
// If it doesn't contain our namespace partition
if (!assignedTopicPartitions.contains(consumerPartitionToUnsubscribe)) {
// For now return false, but maybe we should throw exception?
return false;
}
// Remove it
assignedTopicPartitions.remove(consumerPartitionToUnsubscribe);
// Convert to TopicPartitions to interact with underlying kafka consumer.
Set<TopicPartition> reassignedTopicPartitions = new HashSet<>();
for (ConsumerPartition consumerPartition : assignedTopicPartitions) {
reassignedTopicPartitions.add(new TopicPartition(consumerPartition.namespace(), consumerPartition.partition()));
}
// Reassign consumer
kafkaConsumer.assign(reassignedTopicPartitions);
// return boolean
return true;
}
/**
* Returns what the consumer considers its current "finished" state to be. This means the highest
* offsets for all partitions its consuming that it has tracked as having been complete.
*
* @return The Consumer's current state.
*/
public ConsumerState getCurrentState() {
return partitionOffsetsManager.getCurrentState();
}
/**
* @return returns our unique consumer identifier.
*/
String getConsumerId() {
return getConsumerConfig().getConsumerId();
}
/**
* @return Deserializer instance.
*/
Deserializer getDeserializer() {
return deserializer;
}
/**
* This will remove all state from the persistence manager.
* This is typically called when the consumer has finished reading
* everything that it wants to read, and does not need to be resumed.
*/
public void removeConsumerState() {
logger.info("Removing Consumer state for ConsumerId: {}", getConsumerId());
final Set<ConsumerPartition> consumerPartitions = partitionOffsetsManager.getAllManagedConsumerPartitions();
for (ConsumerPartition consumerPartition : consumerPartitions) {
getPersistenceAdapter().clearConsumerState(getConsumerId(), consumerPartition.partition());
}
}
/**
* Get the partitions that this particular consumer instance should consume from.
* @return List of partitions to consume from
*/
private List<TopicPartition> getPartitions() {
// Ask Kafka for all of the partitions that are available
final List<PartitionInfo> allPartitionInfos = getKafkaConsumer().partitionsFor(consumerConfig.getTopic());
// Convert all of our partition info objects into a primitive list of the partition ids
final int[] allPartitionIds = allPartitionInfos.stream().map(PartitionInfo::partition).mapToInt(i -> i).toArray();
// Perform our calculation
final int[] partitionsIds = PartitionDistributor.calculatePartitionAssignment(
consumerConfig.getNumberOfConsumers(),
consumerConfig.getIndexOfConsumer(),
allPartitionIds
);
// Convert our partition ids back to a list of TopicPartition records
final List<TopicPartition> topicPartitions = Lists.newArrayList();
for (final int partitonId : partitionsIds) {
topicPartitions.add(
new TopicPartition(consumerConfig.getTopic(), partitonId)
);
}
// Return TopicPartitions for our assigned partitions
return topicPartitions;
}
private Map<MetricName, ? extends Metric> metrics() {
return getKafkaConsumer().metrics();
}
/**
* @return The maximum lag of the consumer.
*/
public double getMaxLag() {
for (Map.Entry<MetricName, ? extends Metric> entry : metrics().entrySet()) {
if (entry.getKey().name().equals("records-lag-max")) {
return entry.getValue().value();
}
}
// Fall thru return value?
return -1.0;
}
}