Skip to content

Commit

Permalink
[KafkaIO] Fix potential data race in ReadFromKafkaDoFn.AverageRecordS…
Browse files Browse the repository at this point in the history
…ize (#33073)

* Add comments clarifying offets and record size calculation
  • Loading branch information
sjvanrossum authored Nov 12, 2024
1 parent 785ec07 commit 682eaef
Showing 1 changed file with 38 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.io.kafka.KafkaIO.ReadSourceDescriptors;
import org.apache.beam.sdk.io.kafka.KafkaIOUtils.MovingAvg;
Expand Down Expand Up @@ -338,13 +340,18 @@ public WatermarkEstimator<Instant> newWatermarkEstimator(
public double getSize(
@Element KafkaSourceDescriptor kafkaSourceDescriptor, @Restriction OffsetRange offsetRange)
throws Exception {
// If present, estimates the record size to offset gap ratio. Compacted topics may hold less
// records than the estimated offset range due to record deletion within a partition.
final LoadingCache<TopicPartition, AverageRecordSize> avgRecordSize =
Preconditions.checkStateNotNull(this.avgRecordSize);
double numRecords =
// The tracker estimates the offset range by subtracting the last claimed position from the
// currently observed end offset for the partition belonging to this split.
double estimatedOffsetRange =
restrictionTracker(kafkaSourceDescriptor, offsetRange).getProgress().getWorkRemaining();
// Before processing elements, we don't have a good estimated size of records and offset gap.
// Return the estimated offset range without scaling by a size to gap ratio.
if (!avgRecordSize.asMap().containsKey(kafkaSourceDescriptor.getTopicPartition())) {
return numRecords;
return estimatedOffsetRange;
}
if (offsetEstimatorCache != null) {
for (Map.Entry<TopicPartition, KafkaLatestOffsetEstimator> tp :
Expand All @@ -353,7 +360,12 @@ public double getSize(
}
}

return avgRecordSize.get(kafkaSourceDescriptor.getTopicPartition()).getTotalSize(numRecords);
// When processing elements, a moving average estimates the size of records and offset gap.
// Return the estimated offset range scaled by the estimated size to gap ratio.
return estimatedOffsetRange
* avgRecordSize
.get(kafkaSourceDescriptor.getTopicPartition())
.estimateRecordByteSizeToOffsetCountRatio();
}

@NewTracker
Expand Down Expand Up @@ -665,22 +677,42 @@ private Map<String, Object> overrideBootstrapServersConfig(
return config;
}

// TODO: Collapse the two moving average trackers into a single accumulator using a single Guava
// AtomicDouble. Note that this requires that a single thread will call update and that while get
// may be called by multiple threads the method must only load the accumulator itself.
@ThreadSafe
private static class AverageRecordSize {
@GuardedBy("this")
private MovingAvg avgRecordSize;

@GuardedBy("this")
private MovingAvg avgRecordGap;

public AverageRecordSize() {
this.avgRecordSize = new MovingAvg();
this.avgRecordGap = new MovingAvg();
}

public void update(int recordSize, long gap) {
public synchronized void update(int recordSize, long gap) {
avgRecordSize.update(recordSize);
avgRecordGap.update(gap);
}

public double getTotalSize(double numRecords) {
return avgRecordSize.get() * numRecords / (1 + avgRecordGap.get());
public double estimateRecordByteSizeToOffsetCountRatio() {
double avgRecordSize;
double avgRecordGap;

synchronized (this) {
avgRecordSize = this.avgRecordSize.get();
avgRecordGap = this.avgRecordGap.get();
}

// The offset increases between records in a batch fetched from a compacted topic may be
// greater than 1. Compacted topics only store records with the greatest offset per key per
// partition, the records in between are deleted and will not be observed by a consumer.
// The observed gap between offsets is used to estimate the number of records that are likely
// to be observed for the provided number of records.
return avgRecordSize / (1 + avgRecordGap);
}
}

Expand Down

0 comments on commit 682eaef

Please sign in to comment.