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 @@ -78,7 +78,10 @@
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.Deserializer;
import org.checkerframework.checker.nullness.qual.EnsuresNonNull;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.checkerframework.checker.nullness.qual.RequiresNonNull;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -316,6 +319,16 @@ public Consumer<byte[], byte[]> load(
private final SerializableSupplier<LoadingCache<KafkaSourceDescriptor, Consumer<byte[], byte[]>>>
pollConsumerCacheSupplier;

private transient @MonotonicNonNull LoadingCache<KafkaSourceDescriptor, MovingAvg>
avgRecordSizeCache;

private transient @MonotonicNonNull LoadingCache<
KafkaSourceDescriptor, KafkaLatestOffsetEstimator>
latestOffsetEstimatorCache;

private transient @MonotonicNonNull LoadingCache<KafkaSourceDescriptor, Consumer<byte[], byte[]>>
pollConsumerCache;

// Valid between bundle start and bundle finish.
private transient @Nullable Deserializer<K> keyDeserializerInstance = null;
private transient @Nullable Deserializer<V> valueDeserializerInstance = null;
Expand Down Expand Up @@ -433,9 +446,12 @@ private void refresh() {
}

@GetInitialRestriction
@RequiresNonNull({"pollConsumerCache"})
public OffsetRange initialRestriction(@Element KafkaSourceDescriptor kafkaSourceDescriptor) {
final Consumer<byte[], byte[]> consumer =
pollConsumerCacheSupplier.get().getUnchecked(kafkaSourceDescriptor);
final LoadingCache<KafkaSourceDescriptor, Consumer<byte[], byte[]>> pollConsumerCache =
this.pollConsumerCache;

final Consumer<byte[], byte[]> consumer = pollConsumerCache.getUnchecked(kafkaSourceDescriptor);

final long startOffset;
final long stopOffset;
Expand Down Expand Up @@ -513,12 +529,16 @@ public WatermarkEstimator<Instant> newWatermarkEstimator(
}

@GetSize
@RequiresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache"})
public double getSize(
@Element KafkaSourceDescriptor kafkaSourceDescriptor, @Restriction OffsetRange offsetRange) {
final LoadingCache<KafkaSourceDescriptor, MovingAvg> avgRecordSizeCache =
this.avgRecordSizeCache;

// 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 @Nullable MovingAvg avgRecordSize =
avgRecordSizeCacheSupplier.get().getIfPresent(kafkaSourceDescriptor);
avgRecordSizeCache.getIfPresent(kafkaSourceDescriptor);
// 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.
final double estimatedOffsetRange =
Expand All @@ -533,8 +553,12 @@ public double getSize(
}

@NewTracker
@RequiresNonNull({"latestOffsetEstimatorCache"})
public OffsetRangeTracker restrictionTracker(
@Element KafkaSourceDescriptor kafkaSourceDescriptor, @Restriction OffsetRange restriction) {
final LoadingCache<KafkaSourceDescriptor, KafkaLatestOffsetEstimator>
latestOffsetEstimatorCache = this.latestOffsetEstimatorCache;

if (restriction.getTo() < Long.MAX_VALUE) {
return new OffsetRangeTracker(restriction);
}
Expand All @@ -543,22 +567,28 @@ public OffsetRangeTracker restrictionTracker(
// so we want to minimize the amount of connections that we start and track with Kafka. Another
// point is that it has a memoized backlog, and this should make that more reusable estimations.
return new GrowableOffsetRangeTracker(
restriction.getFrom(),
latestOffsetEstimatorCacheSupplier.get().getUnchecked(kafkaSourceDescriptor));
restriction.getFrom(), latestOffsetEstimatorCache.getUnchecked(kafkaSourceDescriptor));
}

@ProcessElement
@RequiresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache", "pollConsumerCache"})
public ProcessContinuation processElement(
@Element KafkaSourceDescriptor kafkaSourceDescriptor,
RestrictionTracker<OffsetRange, Long> tracker,
WatermarkEstimator<Instant> watermarkEstimator,
MultiOutputReceiver receiver)
throws Exception {
final MovingAvg avgRecordSize = avgRecordSizeCacheSupplier.get().get(kafkaSourceDescriptor);
final LoadingCache<KafkaSourceDescriptor, MovingAvg> avgRecordSizeCache =
this.avgRecordSizeCache;
final LoadingCache<KafkaSourceDescriptor, KafkaLatestOffsetEstimator>
latestOffsetEstimatorCache = this.latestOffsetEstimatorCache;
final LoadingCache<KafkaSourceDescriptor, Consumer<byte[], byte[]>> pollConsumerCache =
this.pollConsumerCache;

final MovingAvg avgRecordSize = avgRecordSizeCache.get(kafkaSourceDescriptor);
final KafkaLatestOffsetEstimator latestOffsetEstimator =
latestOffsetEstimatorCacheSupplier.get().get(kafkaSourceDescriptor);
final Consumer<byte[], byte[]> consumer =
pollConsumerCacheSupplier.get().get(kafkaSourceDescriptor);
latestOffsetEstimatorCache.get(kafkaSourceDescriptor);
final Consumer<byte[], byte[]> consumer = pollConsumerCache.get(kafkaSourceDescriptor);
final Deserializer<K> keyDeserializerInstance =
Preconditions.checkStateNotNull(this.keyDeserializerInstance);
final Deserializer<V> valueDeserializerInstance =
Expand Down Expand Up @@ -734,7 +764,12 @@ public Coder<OffsetRange> restrictionCoder() {
}

@Setup
@EnsuresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache", "pollConsumerCache"})
public void setup() throws Exception {
avgRecordSizeCache = avgRecordSizeCacheSupplier.get();
latestOffsetEstimatorCache = latestOffsetEstimatorCacheSupplier.get();
pollConsumerCache = pollConsumerCacheSupplier.get();

keyDeserializerInstance = keyDeserializerProvider.getDeserializer(consumerConfig, true);
valueDeserializerInstance = valueDeserializerProvider.getDeserializer(consumerConfig, false);
if (checkStopReadingFn != null) {
Expand All @@ -743,7 +778,15 @@ public void setup() throws Exception {
}

@Teardown
@RequiresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache", "pollConsumerCache"})
public void teardown() throws Exception {
final LoadingCache<KafkaSourceDescriptor, MovingAvg> avgRecordSizeCache =
this.avgRecordSizeCache;
final LoadingCache<KafkaSourceDescriptor, KafkaLatestOffsetEstimator>
latestOffsetEstimatorCache = this.latestOffsetEstimatorCache;
final LoadingCache<KafkaSourceDescriptor, Consumer<byte[], byte[]>> pollConsumerCache =
this.pollConsumerCache;

try {
if (valueDeserializerInstance != null) {
Closeables.close(valueDeserializerInstance, true);
Expand All @@ -761,9 +804,9 @@ public void teardown() throws Exception {
}

// Allow the cache to perform clean up tasks when this instance is about to be deleted.
avgRecordSizeCacheSupplier.get().cleanUp();
latestOffsetEstimatorCacheSupplier.get().cleanUp();
pollConsumerCacheSupplier.get().cleanUp();
avgRecordSizeCache.cleanUp();
latestOffsetEstimatorCache.cleanUp();
pollConsumerCache.cleanUp();
}

private static Instant ensureTimestampWithinBounds(Instant timestamp) {
Expand Down
Loading