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
[BEAM-10865] Support for Kafka deserialization API with headers (since Kafka API 2.1.0) #12794
[BEAM-10865] Support for Kafka deserialization API with headers (since Kafka API 2.1.0) #12794
Conversation
Seems to fail the "Run Java PreCommit" check with an unrelated error https://ci-beam.apache.org/job/beam_PreCommit_Java_Commit/13456/testReport/junit/org.apache.beam.sdk.transforms/ParDoLifecycleTest/testTeardownCalledAfterExceptionInFinishBundleStateful/ |
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java
Outdated
Show resolved
Hide resolved
|
||
@Override | ||
public Integer deserialize(String topic, byte[] data) { | ||
StackTraceElement[] stackTraceElements = Thread.currentThread().getStackTrace(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In the build.gradle you'll want to setup a junit test run that uses Kafka 2.1.0 and executes these tests.
This is a pretty good example of how to get this going:
https://docs.gradle.org/current/userguide/java_testing.html#sec:configuring_java_integration_tests
You shouldn't need any additional source sets but you'll want to add a configuration for kafka210 that adds the dependency and also the test task that uses it.
Known issue. |
Reflection might not be that bad enough to warrant an improvement but there is a way to compile code against multiple versions of Kafka using multiple source sets and then at runtime choose which version of the class to use. This would remove a lot of the overhead. (Flink does this in the Flink runner code base) Do you see the reflective calls dominating performance in some way? Also, this can always be improved later so unless its bad I wouldn't worry about it. You can leave a JIRA detailing this and a TODO as comment in the code. |
R: @lukecwik |
…regard it being a default or not
Waiting on tests that use Kafka 2.1.0 explicitly as described in my first review. |
Hi Luke, Apologies for the lag on this. TLDR I have the integration test against Executed with Main test task
Kafka client API 2.1.0 specific task
Any failure in task I opted to run the whole !!! FAIL: However I'm failing at being able to inject the 2.1.0 kafka-client module dependency for the integration test and I'm wondering if you have any obvious thoughts on what I've done wrong in specifying the test module dependency? I have some ideas (based on the python specific tasks and multi-version support) on how to support test tasks for all Kafka client API versions, but beyond the scope of this pull request. |
…rce Kafka clients API 2.1.0
Hi Luke, I got it working with a resolution strategy to force Original Kafka test, default API: API 2.1.0 specific test: I reverse tested with the following corruption of the 2.1.0 path in the diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java
index 9ca5bfc990..37a4a2bc8e 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java
@@ -132,7 +132,7 @@ class ConsumerSpEL {
mapContext.setVariable("deserializer", deserializer);
mapContext.setVariable("topic", rawRecord.topic());
mapContext.setVariable("headers", rawRecord.headers());
- mapContext.setVariable("data", isKey ? rawRecord.key() : rawRecord.value());
+ mapContext.setVariable("data", isKey ? rawRecord.value() : rawRecord.key());
return deserializeWithHeadersExpression.getValue(mapContext);
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Using the forceStrategy makes sense since we have to override the version from BeamModulePlugin.groovy as you noticed with library.java.kafka_clients
.
If you could look into the additional post review comments and address them in a follow-up PR.
outputs.upToDateWhen { false } | ||
testClassesDirs = sourceSets.test.output.classesDirs | ||
classpath = configurations.kafkaVersion210 + sourceSets.test.runtimeClasspath | ||
include '**/KafkaIOTest.class' |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we run all the unit tests?
include '**/KafkaIOTest.class' | |
include '**/KafkaIOTest.class' |
task kafkaVersion210Test(type: Test) { | ||
group = "Verification" | ||
description = 'Runs KafkaIO tests with Kafka clients API 2.1.0' | ||
outputs.upToDateWhen { false } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Instead of doing outputs.upToDateWhen { false }
to have this run every time, please include the set of inputs/outputs that makes sense in a follow-up PR so that an incremental build can be supported.
Thanks for the feedback and the merge Luke. I'll address the updates in a PR. I agree all tests makes more sense as one can never have enough coverage. That said, I'm been thinking about the Thoughts? |
I think that is a good idea to enumerate the most popular kafka client versions. I don't think the unit tests are that slow and the community can always break-up the set that run to use more jenkins executors in parallel if it becomes a large enough issue. |
TLDR Let KafkaIO support the deserializer API with headers
References mailing list posts:
Design decisions
The reason for SpEL is because with kafka-clients API < 2.1.0 as dependency, compilation fails with:
Because the headers default API only landed in 2.1.0 via apache/kafka@f1f7192#diff-a4f4aee88ce5091db576139f6c610ced
I opted for
ConsumerSpEL#deserializeKey
andConsumerSpEL#deserializeValue
as API to ensure forward looking consistency for bothKafkaUnboundedReader
andReadFromKafkaDoFn
as both already depended on an instance thereof.Not so great things
Using the SpEL for kafka-client API 2.1.0 onwards effectively turns the deserialization path into a more expensive indirection by calling the deserializer methods using reflection (2x per record, 1 x key, 1 x value):
And effectively this penalizes the more recent Kafka API versions in favor of the older ones. I have not measured the overhead thereof, yet.
Other avenues explored
For runtime deserialization:
For regression tests (that we don't stop passing headers in the future):
I tried Mockito and Powermock implementations on both
LocalDeserializerProvider
and the Integer and Long serializers in tests, but found the stack to be too deep and backed out of that.Ditto for attempting to spy on
ConsumerRecord#headers()
(expect it to be called twice as much for the newer API), but again deep stack and hard to assert. Just the call is interesting because the constructor used forConsumerRecord
in the tests does not use the one that sets headers, presumably for client API compatibility too.Evaluated
ExtendedSerializer
´s wrapper, butExtendedSerializer
is deprecated API and no point in bringing that in as a dependencyHow the current regression test works
I figured it makes sense given this feature tests deserialization and the whole test suite depends on the
Integer
(for keys) and Long (for values) ones to implement a key and value deserializer that can assert the behaviour. And herein also lies somewhat of a problem because the test case is a bit weak as I relied on stack frames (wide array of suppored client versions makes anything else super complex) to infer the caller of thedeserialize
method, but unfortunately only class and method name context is provide and no arguments size of 3 or even types on those to assert on.Kafka client API 1.0.0 :
For clients before 2.1.0, frame 3 is
ConsumerSpEL#deserializeKey
, meaning it was called directly and not via a default or actual implementation onDeserializer
. Frames 1 and 2 being equal is because of thesuper.deserialize
call.Kafka client API 2.1.0+ :
For clients 2.1.0 and beyond, frame 3 is
org.apache.kafka.common.serialization.Deserializer#deserialize
. This is true for the bundled deserializers used in the tests because they delegate the call to the implementation onDeserializer
. In practice this may refer to an actual override implementation.Feedback items for me
consumer.seekToEnd
andconsumer.assign
are once off / periodic APIs and not called as often as twice per record.Questions
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
R: @username
).[BEAM-XXX] Fixes bug in ApproximateQuantiles
, where you replaceBEAM-XXX
with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.CHANGES.md
with noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
Post-Commit Tests Status (on master branch)
Pre-Commit Tests Status (on master branch)
See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI.