Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

BUG FIX : fix-ReadFromKafkaViaSDF-bug-shall-set-coder #34505

Open
wants to merge 10 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
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
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@

* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
* Fixed read Beam rows from cross-lang transform (for example, ReadFromJdbc) involving negative 32-bit integers incorrectly decoded to large integers ([#34089](https://github.com/apache/beam/issues/34089))
* (Java) Fixed SDF-based KafkaIO (ReadFromKafkaViaSDF) to properly handle custom deserializers that extend Deserializer<Row> interface([#34505](https://github.com/apache/beam/pull/34505))

## Security Fixes
* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)).
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1784,8 +1784,10 @@ public PCollection<KafkaRecord<K, V>> expand(PBegin input) {
.withConsumerConfigOverrides(kafkaRead.getConsumerConfig())
.withOffsetConsumerConfigOverrides(kafkaRead.getOffsetConsumerConfig())
.withConsumerFactoryFn(kafkaRead.getConsumerFactoryFn())
.withKeyDeserializerProvider(kafkaRead.getKeyDeserializerProvider())
.withValueDeserializerProvider(kafkaRead.getValueDeserializerProvider())
.withKeyDeserializerProviderAndCoder(
kafkaRead.getKeyDeserializerProvider(), keyCoder)
.withValueDeserializerProviderAndCoder(
kafkaRead.getValueDeserializerProvider(), valueCoder)
Comment on lines +1787 to +1790
Copy link
Contributor Author

@yyfhust yyfhust Apr 2, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

keyCoder and valueCoder must be non-nullable. they are resolved here

Coder<K> keyCoder = getKeyCoder(coderRegistry);
Coder<V> valueCoder = getValueCoder(coderRegistry);

either from user input or infer from coderegistry

Copy link
Contributor Author

@yyfhust yyfhust Apr 2, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is at the entry of kafka IO, and it will 100% return a coder :

private Coder<V> getValueCoder(CoderRegistry coderRegistry) {
return (getValueCoder() != null)
? getValueCoder()
: Preconditions.checkStateNotNull(getValueDeserializerProvider()).getCoder(coderRegistry);

if coder is given, it will return the coder specified by user. If not, it will attempt to retrieve a coder from registry , which only has coder for build-in deserializer.

.withManualWatermarkEstimator()
.withTimestampPolicyFactory(kafkaRead.getTimestampPolicyFactory())
.withCheckStopReadingFn(kafkaRead.getCheckStopReadingFn())
Expand Down Expand Up @@ -2363,16 +2365,6 @@ public ReadSourceDescriptors<K, V> withBootstrapServers(String bootstrapServers)
ImmutableMap.of(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers));
}

public ReadSourceDescriptors<K, V> withKeyDeserializerProvider(
@Nullable DeserializerProvider<K> deserializerProvider) {
return toBuilder().setKeyDeserializerProvider(deserializerProvider).build();
}

public ReadSourceDescriptors<K, V> withValueDeserializerProvider(
@Nullable DeserializerProvider<V> deserializerProvider) {
return toBuilder().setValueDeserializerProvider(deserializerProvider).build();
}

/**
* Sets a Kafka {@link Deserializer} to interpret key bytes read from Kafka.
*
Expand All @@ -2386,6 +2378,31 @@ public ReadSourceDescriptors<K, V> withKeyDeserializer(
return withKeyDeserializerProvider(LocalDeserializerProvider.of(keyDeserializer));
}

/**
* Sets a Kafka {@link Deserializer} for interpreting key bytes read from Kafka along with a
* {@link Coder} for helping the Beam runner materialize key objects at runtime if necessary.
*
* <p>Use this method to override the coder inference performed within {@link
* #withKeyDeserializer(Class)}.
*/
public ReadSourceDescriptors<K, V> withKeyDeserializerAndCoder(
Class<? extends Deserializer<K>> keyDeserializer, Coder<K> keyCoder) {
return withKeyDeserializer(keyDeserializer).toBuilder().setKeyCoder(keyCoder).build();
}

public ReadSourceDescriptors<K, V> withKeyDeserializerProvider(
@Nullable DeserializerProvider<K> deserializerProvider) {
return toBuilder().setKeyDeserializerProvider(deserializerProvider).build();
}

public ReadSourceDescriptors<K, V> withKeyDeserializerProviderAndCoder(
@Nullable DeserializerProvider<K> deserializerProvider, Coder<K> keyCoder) {
return toBuilder()
.setKeyDeserializerProvider(deserializerProvider)
.setKeyCoder(keyCoder)
.build();
}

/**
* Sets a Kafka {@link Deserializer} to interpret value bytes read from Kafka.
*
Expand All @@ -2399,18 +2416,6 @@ public ReadSourceDescriptors<K, V> withValueDeserializer(
return withValueDeserializerProvider(LocalDeserializerProvider.of(valueDeserializer));
}

/**
* Sets a Kafka {@link Deserializer} for interpreting key bytes read from Kafka along with a
* {@link Coder} for helping the Beam runner materialize key objects at runtime if necessary.
*
* <p>Use this method to override the coder inference performed within {@link
* #withKeyDeserializer(Class)}.
*/
public ReadSourceDescriptors<K, V> withKeyDeserializerAndCoder(
Class<? extends Deserializer<K>> keyDeserializer, Coder<K> keyCoder) {
return withKeyDeserializer(keyDeserializer).toBuilder().setKeyCoder(keyCoder).build();
}

/**
* Sets a Kafka {@link Deserializer} for interpreting value bytes read from Kafka along with a
* {@link Coder} for helping the Beam runner materialize value objects at runtime if necessary.
Expand All @@ -2423,6 +2428,19 @@ public ReadSourceDescriptors<K, V> withValueDeserializerAndCoder(
return withValueDeserializer(valueDeserializer).toBuilder().setValueCoder(valueCoder).build();
}

public ReadSourceDescriptors<K, V> withValueDeserializerProvider(
@Nullable DeserializerProvider<V> deserializerProvider) {
return toBuilder().setValueDeserializerProvider(deserializerProvider).build();
}

public ReadSourceDescriptors<K, V> withValueDeserializerProviderAndCoder(
@Nullable DeserializerProvider<V> deserializerProvider, Coder<V> valueCoder) {
return toBuilder()
.setValueDeserializerProvider(deserializerProvider)
.setValueCoder(valueCoder)
.build();
}

/**
* A factory to create Kafka {@link Consumer} from consumer configuration. This is useful for
* supporting another version of Kafka consumer. Default is {@link KafkaConsumer}.
Expand Down
Loading