Skip to content

Commit c8786f0

Browse files
committed
NIFI-15563 Report LAG in records from ConsumeKafka processor
1 parent 8506a91 commit c8786f0

File tree

2 files changed

+93
-1
lines changed
  • nifi-extension-bundles/nifi-kafka-bundle

2 files changed

+93
-1
lines changed

nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-3-integration/src/test/java/org/apache/nifi/kafka/processors/ConsumeKafkaIT.java

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import org.apache.nifi.controller.ControllerService;
2323
import org.apache.nifi.kafka.processors.consumer.ProcessingStrategy;
2424
import org.apache.nifi.kafka.service.Kafka3ConnectionService;
25+
import org.apache.nifi.kafka.service.api.common.TopicPartitionSummary;
2526
import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset;
2627
import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
2728
import org.apache.nifi.reporting.InitializationException;
@@ -40,6 +41,7 @@
4041
import java.util.List;
4142
import java.util.UUID;
4243
import java.util.concurrent.ExecutionException;
44+
import java.util.stream.IntStream;
4345

4446
import static org.junit.jupiter.api.Assertions.assertEquals;
4547
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -254,6 +256,39 @@ public void testConsumesAllRecordsWithoutDuplicates() throws ExecutionException,
254256
runner.assertAllFlowFilesTransferred(ConsumeKafka.SUCCESS, 1);
255257
}
256258

259+
@Test
260+
@Timeout(30)
261+
public void testLagReporting() throws ExecutionException, InterruptedException {
262+
final String topic = "testLagReporting";
263+
final int partition = 0;
264+
final int maxPollRecords = 2;
265+
266+
runner.setProperty(ConsumeKafka.GROUP_ID, "testLagReportingGroup");
267+
runner.setProperty(ConsumeKafka.TOPICS, topic);
268+
runner.setProperty(ConsumeKafka.PROCESSING_STRATEGY, ProcessingStrategy.FLOW_FILE.getValue());
269+
runner.setProperty(ConsumeKafka.MAX_UNCOMMITTED_TIME, "1000 millis");
270+
271+
final ControllerService connectionService = runner.getControllerService(CONNECTION_SERVICE_ID);
272+
runner.disableControllerService(connectionService);
273+
runner.setProperty(connectionService, Kafka3ConnectionService.MAX_POLL_RECORDS, Integer.toString(maxPollRecords));
274+
runner.enableControllerService(connectionService);
275+
276+
277+
List<ProducerRecord<String, String>> records = IntStream.range(0, 1000)
278+
.mapToObj(i -> new ProducerRecord<String, String>(topic, partition, "key" + i, "val" + i))
279+
.toList();
280+
produce(topic, records);
281+
282+
final TopicPartitionSummary tps = new TopicPartitionSummary(topic, partition);
283+
final String lagMetricName = ((ConsumeKafka) runner.getProcessor()).makeLagMetricName(tps);
284+
runner.run(1, false, true);
285+
286+
// consume until at least one gauge with lag gets recorded
287+
while (runner.getGaugeValues(lagMetricName).isEmpty()) {
288+
runner.run(1, false, false);
289+
}
290+
}
291+
257292
@Timeout(5)
258293
@Test
259294
void testMaxUncommittedSize() throws InterruptedException, ExecutionException {

nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java

Lines changed: 58 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import org.apache.nifi.kafka.processors.consumer.convert.WrapperRecordStreamKafkaMessageConverter;
4242
import org.apache.nifi.kafka.service.api.KafkaConnectionService;
4343
import org.apache.nifi.kafka.service.api.common.PartitionState;
44+
import org.apache.nifi.kafka.service.api.common.TopicPartitionSummary;
4445
import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset;
4546
import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
4647
import org.apache.nifi.kafka.service.api.consumer.PollingContext;
@@ -57,6 +58,7 @@
5758
import org.apache.nifi.processor.Relationship;
5859
import org.apache.nifi.processor.VerifiableProcessor;
5960
import org.apache.nifi.processor.exception.ProcessException;
61+
import org.apache.nifi.processor.metrics.CommitTiming;
6062
import org.apache.nifi.processor.util.StandardValidators;
6163
import org.apache.nifi.serialization.RecordReaderFactory;
6264
import org.apache.nifi.serialization.RecordSetWriterFactory;
@@ -68,15 +70,19 @@
6870
import java.time.Duration;
6971
import java.util.ArrayList;
7072
import java.util.Collection;
73+
import java.util.HashSet;
7174
import java.util.Iterator;
7275
import java.util.List;
7376
import java.util.Map;
77+
import java.util.OptionalLong;
7478
import java.util.Queue;
7579
import java.util.Set;
7680
import java.util.concurrent.LinkedBlockingQueue;
7781
import java.util.concurrent.TimeUnit;
7882
import java.util.concurrent.atomic.AtomicInteger;
83+
import java.util.function.Function;
7984
import java.util.regex.Pattern;
85+
import java.util.stream.Collectors;
8086

8187
import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
8288

@@ -432,7 +438,9 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
432438
break;
433439
}
434440

435-
final Iterator<ByteRecord> consumerRecords = consumerService.poll(maxWaitDuration).iterator();
441+
final TopicPartitionScanningIterator consumerRecords =
442+
new TopicPartitionScanningIterator(consumerService.poll(maxWaitDuration).iterator());
443+
436444
if (!consumerRecords.hasNext()) {
437445
getLogger().trace("No Kafka Records consumed: {}", pollingContext);
438446
// 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
445453

446454
recordsReceived = true;
447455
processConsumerRecords(context, session, offsetTracker, consumerRecords);
456+
reportCurrentLag(consumerService, session, consumerRecords.getTopicPartitionSummaries());
448457

449458
// Check if a rebalance occurred during poll - if so, break to commit what we have
450459
if (consumerService.hasRevokedPartitions()) {
@@ -498,6 +507,27 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
498507
});
499508
}
500509

510+
private void reportCurrentLag(final KafkaConsumerService consumerService, final ProcessSession session, final Set<TopicPartitionSummary> topicPartitionSummaries) {
511+
Map<TopicPartitionSummary, OptionalLong> topicPartitionLag =
512+
topicPartitionSummaries.stream()
513+
.map(ps -> new TopicPartitionSummary(ps.getTopic(), ps.getPartition()))
514+
.collect(Collectors.toMap(
515+
Function.identity(),
516+
tps -> consumerService.currentLag(tps)
517+
));
518+
519+
topicPartitionLag.forEach((tps, lag) -> {
520+
if (lag.isPresent()) {
521+
final String gaugeName = makeLagMetricName(tps);
522+
session.recordGauge(gaugeName, lag.getAsLong(), CommitTiming.NOW);
523+
}
524+
});
525+
}
526+
527+
String makeLagMetricName(final TopicPartitionSummary tps) {
528+
return tps.getTopic() + "_" + tps.getPartition() + "_currentLag";
529+
}
530+
501531
private void commitOffsets(final KafkaConsumerService consumerService, final OffsetTracker offsetTracker, final PollingContext pollingContext, final ProcessSession session) {
502532
try {
503533
if (commitOffsets) {
@@ -686,4 +716,31 @@ private PollingContext createPollingContext(final ProcessContext context) {
686716

687717
return pollingContext;
688718
}
719+
720+
static class TopicPartitionScanningIterator implements Iterator<ByteRecord> {
721+
722+
private final Iterator<ByteRecord> delegate;
723+
private final Set<TopicPartitionSummary> topicPartitionSummaries = new HashSet<>();
724+
725+
TopicPartitionScanningIterator(Iterator<ByteRecord> delegate) {
726+
this.delegate = delegate;
727+
}
728+
729+
@Override
730+
public boolean hasNext() {
731+
return delegate.hasNext();
732+
}
733+
734+
@Override
735+
public ByteRecord next() {
736+
ByteRecord record = delegate.next();
737+
topicPartitionSummaries.add(new TopicPartitionSummary(record.getTopic(), record.getPartition()));
738+
return record;
739+
740+
}
741+
742+
public Set<TopicPartitionSummary> getTopicPartitionSummaries() {
743+
return topicPartitionSummaries;
744+
}
745+
}
689746
}

0 commit comments

Comments
 (0)