Skip to content

Commit 010a17c

Browse files
author
Hao Xu
committed
Adding poll rate to see how slow partition consumping.
1 parent 5dfe33b commit 010a17c

File tree

4 files changed

+61
-6
lines changed

4 files changed

+61
-6
lines changed

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ConsumptionTask.java

Lines changed: 38 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
import com.linkedin.venice.pubsub.api.PubSubTopicPartition;
1111
import com.linkedin.venice.utils.ExceptionUtils;
1212
import com.linkedin.venice.utils.LatencyUtils;
13+
import com.linkedin.venice.utils.RedundantExceptionFilter;
1314
import com.linkedin.venice.utils.Utils;
1415
import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap;
1516
import io.tehuti.metrics.MetricConfig;
@@ -19,6 +20,7 @@
1920
import java.util.List;
2021
import java.util.Map;
2122
import java.util.Set;
23+
import java.util.function.Function;
2224
import java.util.function.IntConsumer;
2325
import java.util.function.Supplier;
2426
import 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);

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@
3737
import java.util.concurrent.Executors;
3838
import java.util.concurrent.TimeUnit;
3939
import java.util.concurrent.locks.ReentrantLock;
40+
import java.util.function.Function;
4041
import java.util.function.IntConsumer;
4142
import java.util.function.LongSupplier;
4243
import java.util.function.Supplier;
@@ -90,7 +91,7 @@ public abstract class KafkaConsumerService extends AbstractKafkaConsumerService
9091
private static final int SHUTDOWN_TIMEOUT_IN_SECOND = 1;
9192
// 4MB bitset size, 2 bitmaps for active and old bitset
9293
private static final RedundantExceptionFilter REDUNDANT_LOGGING_FILTER =
93-
new RedundantExceptionFilter(8 * 1024 * 1024 * 4, TimeUnit.MINUTES.toMillis(10));
94+
new RedundantExceptionFilter(8 * 1024 * 1024 * 4, TimeUnit.MINUTES.toMillis(1));
9495

9596
/**
9697
* @param statsOverride injection of stats, for test purposes
@@ -167,6 +168,7 @@ protected KafkaConsumerService(
167168
pubSubConsumer::batchUnsubscribe,
168169
time);
169170

171+
Function<PubSubTopicPartition, Long> offsetLagGetter = pubSubConsumer::getOffsetLag;
170172
ConsumptionTask consumptionTask = new ConsumptionTask(
171173
consumerNamePrefix,
172174
i,
@@ -175,7 +177,9 @@ protected KafkaConsumerService(
175177
bandwidthThrottlerFunction,
176178
recordsThrottlerFunction,
177179
this.aggStats,
178-
cleaner);
180+
cleaner,
181+
offsetLagGetter,
182+
REDUNDANT_LOGGING_FILTER);
179183
consumerToConsumptionTask.putByIndex(pubSubConsumer, consumptionTask, i);
180184
consumerToLocks.put(pubSubConsumer, new ReentrantLock());
181185
}
@@ -560,6 +564,7 @@ private Map<PubSubTopicPartition, TopicPartitionIngestionInfo> getIngestionInfoF
560564
long latestOffset = consumer.getLatestOffset(topicPartition);
561565
double msgRate = consumptionTask.getMessageRate(topicPartition);
562566
double byteRate = consumptionTask.getByteRate(topicPartition);
567+
double pollRate = consumptionTask.getPollRate(topicPartition);
563568
long lastSuccessfulPollTimestamp = consumptionTask.getLastSuccessfulPollTimestamp(topicPartition);
564569
long elapsedTimeSinceLastPollInMs = ConsumptionTask.DEFAULT_TOPIC_PARTITION_NO_POLL_TIMESTAMP;
565570
if (lastSuccessfulPollTimestamp != ConsumptionTask.DEFAULT_TOPIC_PARTITION_NO_POLL_TIMESTAMP) {
@@ -573,6 +578,7 @@ private Map<PubSubTopicPartition, TopicPartitionIngestionInfo> getIngestionInfoF
573578
offsetLag,
574579
msgRate,
575580
byteRate,
581+
pollRate,
576582
consumerIdStr,
577583
elapsedTimeSinceLastPollInMs,
578584
destinationVersionTopicName);

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/TopicPartitionIngestionInfo.java

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@ public class TopicPartitionIngestionInfo {
99
private long offsetLag;
1010
private double msgRate;
1111
private double byteRate;
12+
private double pollRate;
1213
private String consumerIdStr;
1314
private long elapsedTimeSinceLastPollInMs;
1415

@@ -20,13 +21,15 @@ public TopicPartitionIngestionInfo(
2021
@JsonProperty("offsetLag") long offsetLag,
2122
@JsonProperty("msgRate") double msgRate,
2223
@JsonProperty("byteRate") double byteRate,
24+
@JsonProperty("pollRate") double pollRate,
2325
@JsonProperty("consumerIdStr") String consumerIdStr,
2426
@JsonProperty("elapsedTimeSinceLastPollInMs") long elapsedTimeSinceLastPollInMs,
2527
@JsonProperty("versionTopicName") String versionTopicName) {
2628
this.latestOffset = latestOffset;
2729
this.offsetLag = offsetLag;
2830
this.msgRate = msgRate;
2931
this.byteRate = byteRate;
32+
this.pollRate = pollRate;
3033
this.consumerIdStr = consumerIdStr;
3134
this.elapsedTimeSinceLastPollInMs = elapsedTimeSinceLastPollInMs;
3235
this.versionTopicName = versionTopicName;
@@ -60,6 +63,14 @@ public void setByteRate(double byteRate) {
6063
this.byteRate = byteRate;
6164
}
6265

66+
public double getPollRate() {
67+
return pollRate;
68+
}
69+
70+
public void setPollRate(double pollRate) {
71+
this.pollRate = pollRate;
72+
}
73+
6374
public String getConsumerIdStr() {
6475
return consumerIdStr;
6576
}
@@ -97,6 +108,7 @@ public boolean equals(Object o) {
97108
&& this.offsetLag == topicPartitionIngestionInfo.getOffsetLag()
98109
&& Double.doubleToLongBits(this.msgRate) == Double.doubleToLongBits(topicPartitionIngestionInfo.getMsgRate())
99110
&& Double.doubleToLongBits(this.byteRate) == Double.doubleToLongBits(topicPartitionIngestionInfo.getByteRate())
111+
&& Double.doubleToLongBits(this.pollRate) == Double.doubleToLongBits(topicPartitionIngestionInfo.getPollRate())
100112
&& this.consumerIdStr.equals(topicPartitionIngestionInfo.getConsumerIdStr())
101113
&& this.elapsedTimeSinceLastPollInMs == topicPartitionIngestionInfo.getElapsedTimeSinceLastPollInMs()
102114
&& this.versionTopicName.equals(topicPartitionIngestionInfo.getVersionTopicName());
@@ -108,6 +120,7 @@ public int hashCode() {
108120
result = 31 * result + Long.hashCode(offsetLag);
109121
result = 31 * result + Double.hashCode(msgRate);
110122
result = 31 * result + Double.hashCode(byteRate);
123+
result = 31 * result + Double.hashCode(pollRate);
111124
result = 31 * result + consumerIdStr.hashCode();
112125
result = 31 * result + Long.hashCode(elapsedTimeSinceLastPollInMs);
113126
result = 31 * result + versionTopicName.hashCode();
@@ -117,7 +130,7 @@ public int hashCode() {
117130
@Override
118131
public String toString() {
119132
return "{" + "latestOffset:" + latestOffset + ", offsetLag:" + offsetLag + ", msgRate:" + msgRate + ", byteRate:"
120-
+ byteRate + ", consumerIdStr:" + consumerIdStr + ", elapsedTimeSinceLastPollInMs:"
133+
+ byteRate + ", pollRate:" + pollRate + ", consumerIdStr:" + consumerIdStr + ", elapsedTimeSinceLastPollInMs:"
121134
+ elapsedTimeSinceLastPollInMs + ", versionTopicName:" + versionTopicName + '}';
122135
}
123136
}

clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/TopicPartitionIngestionInfoTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ public class TopicPartitionIngestionInfoTest {
2020
public void testJsonParse() throws Exception {
2121
PubSubTopic versionTopic = pubSubTopicRepository.getTopic("test_store_v1");
2222
TopicPartitionIngestionInfo topicPartitionIngestionInfo =
23-
new TopicPartitionIngestionInfo(0, 1, 2.0, 4.0, "consumerIdStr", 7, versionTopic.getName());
23+
new TopicPartitionIngestionInfo(0, 1, 2.0, 4.0, 1.0, "consumerIdStr", 7, versionTopic.getName());
2424
String kafkaUrl = "localhost:1234";
2525
PubSubTopicPartition pubSubTopicPartition = new PubSubTopicPartitionImpl(versionTopic, 0);
2626
Map<String, Map<String, TopicPartitionIngestionInfo>> topicPartitionIngestionContext = new HashMap<>();

0 commit comments

Comments
 (0)