-
Notifications
You must be signed in to change notification settings - Fork 2.9k
NIFI-15563 Report LAG in records from ConsumeKafka processor #10880
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -41,6 +41,7 @@ | |
| import org.apache.nifi.kafka.processors.consumer.convert.WrapperRecordStreamKafkaMessageConverter; | ||
| import org.apache.nifi.kafka.service.api.KafkaConnectionService; | ||
| import org.apache.nifi.kafka.service.api.common.PartitionState; | ||
| import org.apache.nifi.kafka.service.api.common.TopicPartitionSummary; | ||
| import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset; | ||
| import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService; | ||
| import org.apache.nifi.kafka.service.api.consumer.PollingContext; | ||
|
|
@@ -57,6 +58,7 @@ | |
| import org.apache.nifi.processor.Relationship; | ||
| import org.apache.nifi.processor.VerifiableProcessor; | ||
| import org.apache.nifi.processor.exception.ProcessException; | ||
| import org.apache.nifi.processor.metrics.CommitTiming; | ||
| import org.apache.nifi.processor.util.StandardValidators; | ||
| import org.apache.nifi.serialization.RecordReaderFactory; | ||
| import org.apache.nifi.serialization.RecordSetWriterFactory; | ||
|
|
@@ -68,15 +70,19 @@ | |
| import java.time.Duration; | ||
| import java.util.ArrayList; | ||
| import java.util.Collection; | ||
| import java.util.HashSet; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.OptionalLong; | ||
| import java.util.Queue; | ||
| import java.util.Set; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.function.Function; | ||
| import java.util.regex.Pattern; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import static org.apache.nifi.expression.ExpressionLanguageScope.NONE; | ||
|
|
||
|
|
@@ -432,7 +438,9 @@ public void onTrigger(final ProcessContext context, final ProcessSession session | |
| break; | ||
| } | ||
|
|
||
| final Iterator<ByteRecord> consumerRecords = consumerService.poll(maxWaitDuration).iterator(); | ||
| final TopicPartitionScanningIterator consumerRecords = | ||
| new TopicPartitionScanningIterator(consumerService.poll(maxWaitDuration).iterator()); | ||
|
|
||
| if (!consumerRecords.hasNext()) { | ||
| getLogger().trace("No Kafka Records consumed: {}", pollingContext); | ||
| // Check if a rebalance occurred during poll - if so, break to commit what we have | ||
|
|
@@ -445,6 +453,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session | |
|
|
||
| recordsReceived = true; | ||
| processConsumerRecords(context, session, offsetTracker, consumerRecords); | ||
| reportCurrentLag(consumerService, session, consumerRecords.getTopicPartitionSummaries()); | ||
|
|
||
| // Check if a rebalance occurred during poll - if so, break to commit what we have | ||
| if (consumerService.hasRevokedPartitions()) { | ||
|
|
@@ -498,6 +507,27 @@ public void onTrigger(final ProcessContext context, final ProcessSession session | |
| }); | ||
| } | ||
|
|
||
| private void reportCurrentLag(final KafkaConsumerService consumerService, final ProcessSession session, final Set<TopicPartitionSummary> topicPartitionSummaries) { | ||
| Map<TopicPartitionSummary, OptionalLong> topicPartitionLag = | ||
| topicPartitionSummaries.stream() | ||
| .map(ps -> new TopicPartitionSummary(ps.getTopic(), ps.getPartition())) | ||
| .collect(Collectors.toMap( | ||
| Function.identity(), | ||
| tps -> consumerService.currentLag(tps) | ||
| )); | ||
|
|
||
| topicPartitionLag.forEach((tps, lag) -> { | ||
| if (lag.isPresent()) { | ||
| final String gaugeName = makeLagMetricName(tps); | ||
| session.recordGauge(gaugeName, lag.getAsLong(), CommitTiming.NOW); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| String makeLagMetricName(final TopicPartitionSummary tps) { | ||
| return "consume.kafka." + tps.getTopic() + "." + tps.getPartition() + ".currentLag"; | ||
|
||
| } | ||
|
|
||
| private void commitOffsets(final KafkaConsumerService consumerService, final OffsetTracker offsetTracker, final PollingContext pollingContext, final ProcessSession session) { | ||
| try { | ||
| if (commitOffsets) { | ||
|
|
@@ -686,4 +716,31 @@ private PollingContext createPollingContext(final ProcessContext context) { | |
|
|
||
| return pollingContext; | ||
| } | ||
|
|
||
| static class TopicPartitionScanningIterator implements Iterator<ByteRecord> { | ||
|
|
||
| private final Iterator<ByteRecord> delegate; | ||
| private final Set<TopicPartitionSummary> topicPartitionSummaries = new HashSet<>(); | ||
|
|
||
| TopicPartitionScanningIterator(Iterator<ByteRecord> delegate) { | ||
| this.delegate = delegate; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean hasNext() { | ||
| return delegate.hasNext(); | ||
| } | ||
|
|
||
| @Override | ||
| public ByteRecord next() { | ||
| ByteRecord record = delegate.next(); | ||
| topicPartitionSummaries.add(new TopicPartitionSummary(record.getTopic(), record.getPartition())); | ||
|
||
| return record; | ||
|
|
||
| } | ||
|
|
||
| public Set<TopicPartitionSummary> getTopicPartitionSummaries() { | ||
| return topicPartitionSummaries; | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This functional approach results in creating an unnecessary intermediate Map. I recommend rewriting this using traditional for-each loops and avoiding the intermediate Map creation.