Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.beam.sdk.io.kafka.KafkaUnboundedReader.TimestampPolicyContext;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Gauge;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.SerializableFunction;
Expand Down Expand Up @@ -222,6 +223,9 @@ private ReadFromKafkaDoFn(

private transient @Nullable LoadingCache<TopicPartition, AverageRecordSize> avgRecordSize;
private static final long DEFAULT_KAFKA_POLL_TIMEOUT = 2L;

private HashMap<String, Long> perPartitionBacklogMetrics = new HashMap<String, Long>();;

@VisibleForTesting final long consumerPollingTimeout;
@VisibleForTesting final DeserializerProvider<K> keyDeserializerProvider;
@VisibleForTesting final DeserializerProvider<V> valueDeserializerProvider;
Expand Down Expand Up @@ -342,6 +346,13 @@ public double getSize(
if (!avgRecordSize.asMap().containsKey(kafkaSourceDescriptor.getTopicPartition())) {
return numRecords;
}
if (offsetEstimatorCache != null) {
for (Map.Entry<TopicPartition, KafkaLatestOffsetEstimator> tp :
offsetEstimatorCache.entrySet()) {
perPartitionBacklogMetrics.put(tp.getKey().toString(), tp.getValue().estimate());
}
}

return avgRecordSize.get(kafkaSourceDescriptor.getTopicPartition()).getTotalSize(numRecords);
}

Expand Down Expand Up @@ -394,6 +405,13 @@ public ProcessContinuation processElement(
Metrics.distribution(
METRIC_NAMESPACE,
RAW_SIZE_METRIC_PREFIX + kafkaSourceDescriptor.getTopicPartition().toString());
for (Map.Entry<String, Long> backlogSplit : perPartitionBacklogMetrics.entrySet()) {
Gauge backlog =
Metrics.gauge(
METRIC_NAMESPACE, RAW_SIZE_METRIC_PREFIX + "backlogBytes_" + backlogSplit.getKey());
backlog.set(backlogSplit.getValue());
}

// Stop processing current TopicPartition when it's time to stop.
if (checkStopReadingFn != null
&& checkStopReadingFn.apply(kafkaSourceDescriptor.getTopicPartition())) {
Expand Down