1010import com .linkedin .venice .pubsub .api .PubSubTopicPartition ;
1111import com .linkedin .venice .utils .ExceptionUtils ;
1212import com .linkedin .venice .utils .LatencyUtils ;
13+ import com .linkedin .venice .utils .RedundantExceptionFilter ;
1314import com .linkedin .venice .utils .Utils ;
1415import com .linkedin .venice .utils .concurrent .VeniceConcurrentHashMap ;
1516import io .tehuti .metrics .MetricConfig ;
1920import java .util .List ;
2021import java .util .Map ;
2122import java .util .Set ;
23+ import java .util .function .Function ;
2224import java .util .function .IntConsumer ;
2325import java .util .function .Supplier ;
2426import org .apache .logging .log4j .LogManager ;
@@ -49,6 +51,8 @@ class ConsumptionTask implements Runnable {
4951 new VeniceConcurrentHashMap <>();
5052 private final long readCycleDelayMs ;
5153 private final Supplier <Map <PubSubTopicPartition , List <PubSubMessage <KafkaKey , KafkaMessageEnvelope , Long >>>> pollFunction ;
54+
55+ private final Function <PubSubTopicPartition , Long > offsetLagGetter ;
5256 private final IntConsumer bandwidthThrottler ;
5357 private final IntConsumer recordsThrottler ;
5458 private final AggKafkaConsumerServiceStats aggStats ;
@@ -61,6 +65,8 @@ class ConsumptionTask implements Runnable {
6165 */
6266 private final Map <PubSubTopicPartition , Rate > messageRatePerTopicPartition = new VeniceConcurrentHashMap <>();
6367 private final Map <PubSubTopicPartition , Rate > bytesRatePerTopicPartition = new VeniceConcurrentHashMap <>();
68+ private final Map <PubSubTopicPartition , Rate > pollRatePerTopicPartition = new VeniceConcurrentHashMap <>();
69+ private final RedundantExceptionFilter redundantExceptionFilter ;
6470 private final Map <PubSubTopicPartition , Long > lastSuccessfulPollTimestampPerTopicPartition =
6571 new VeniceConcurrentHashMap <>();
6672
@@ -87,14 +93,18 @@ public ConsumptionTask(
8793 final IntConsumer bandwidthThrottler ,
8894 final IntConsumer recordsThrottler ,
8995 final AggKafkaConsumerServiceStats aggStats ,
90- final ConsumerSubscriptionCleaner cleaner ) {
96+ final ConsumerSubscriptionCleaner cleaner ,
97+ Function <PubSubTopicPartition , Long > offsetLagGetter ,
98+ RedundantExceptionFilter redundantExceptionFilter ) {
9199 this .readCycleDelayMs = readCycleDelayMs ;
92100 this .pollFunction = pollFunction ;
93101 this .bandwidthThrottler = bandwidthThrottler ;
94102 this .recordsThrottler = recordsThrottler ;
95103 this .aggStats = aggStats ;
96104 this .cleaner = cleaner ;
97105 this .taskId = taskId ;
106+ this .offsetLagGetter = offsetLagGetter ;
107+ this .redundantExceptionFilter = redundantExceptionFilter ;
98108 this .consumptionTaskIdStr = Utils .getSanitizedStringForLogger (consumerNamePrefix ) + " - " + taskId ;
99109 this .LOGGER = LogManager .getLogger (getClass ().getSimpleName () + "[ " + consumptionTaskIdStr + " ]" );
100110 }
@@ -180,8 +190,11 @@ public void run() {
180190 bytesRatePerTopicPartition
181191 .computeIfAbsent (pubSubTopicPartition , tp -> createRate (lastSuccessfulPollTimestamp ))
182192 .record (payloadSizePerTopicPartition , lastSuccessfulPollTimestamp );
183-
193+ pollRatePerTopicPartition
194+ .computeIfAbsent (pubSubTopicPartition , tp -> createRate (lastSuccessfulPollTimestamp ))
195+ .record (1 , lastSuccessfulPollTimestamp );
184196 consumedDataReceiver .write (topicPartitionMessages );
197+ checkSlowPartitionWithHighLag (pubSubTopicPartition );
185198 }
186199 aggStats .recordTotalConsumerRecordsProducingToWriterBufferLatency (
187200 LatencyUtils .getElapsedTimeFromMsToMs (beforeProducingToWriteBufferTimestamp ));
@@ -278,6 +291,29 @@ Double getByteRate(PubSubTopicPartition topicPartition) {
278291 return 0.0D ;
279292 }
280293
294+ Double getPollRate (PubSubTopicPartition topicPartition ) {
295+ if (pollRatePerTopicPartition .containsKey (topicPartition )) {
296+ return pollRatePerTopicPartition .get (topicPartition ).measure (metricConfig , System .currentTimeMillis ());
297+ }
298+ return 0.0D ;
299+ }
300+
301+ private void checkSlowPartitionWithHighLag (PubSubTopicPartition pubSubTopicPartition ) {
302+ Long offsetLag = offsetLagGetter .apply (pubSubTopicPartition );
303+ Double messageRate = getMessageRate (pubSubTopicPartition );
304+ Double pollRate = getPollRate (pubSubTopicPartition );
305+ String slowTaskWithPartitionStr = consumptionTaskIdStr + " - " + pubSubTopicPartition ;
306+ if (offsetLag > 200000 && messageRate < 200
307+ && !redundantExceptionFilter .isRedundantException (slowTaskWithPartitionStr )) {
308+ LOGGER .warn (
309+ "Slow partition with high lag detected: {}. Lag: {}, Message Rate: {}, Poll Rate: {}" ,
310+ pubSubTopicPartition ,
311+ offsetLag ,
312+ messageRate ,
313+ pollRate );
314+ }
315+ }
316+
281317 PubSubTopic getDestinationIdentifier (PubSubTopicPartition topicPartition ) {
282318 ConsumedDataReceiver <List <PubSubMessage <KafkaKey , KafkaMessageEnvelope , Long >>> dataReceiver =
283319 dataReceiverMap .get (topicPartition );
0 commit comments