kennknowles commented on code in PR #31682:
URL: https://github.com/apache/beam/pull/31682#discussion_r1725286758
##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java:
##########
@@ -2684,26 +2687,49 @@ public PCollection<KafkaRecord<K, V>>
expand(PCollection<KafkaSourceDescriptor>
.getSchemaRegistry()
.getSchemaCoder(KafkaSourceDescriptor.class),
recordCoder));
+
if (isCommitOffsetEnabled() && !configuredKafkaCommit() &&
!isRedistribute()) {
- outputWithDescriptor =
- outputWithDescriptor
- .apply(Reshuffle.viaRandomKey())
- .setCoder(
- KvCoder.of(
- input
- .getPipeline()
- .getSchemaRegistry()
- .getSchemaCoder(KafkaSourceDescriptor.class),
- recordCoder));
-
- PCollection<Void> unused = outputWithDescriptor.apply(new
KafkaCommitOffset<K, V>(this));
- unused.setCoder(VoidCoder.of());
+ // Add transform for committing offsets to Kafka with consistency
with beam pipeline data
+ // processing.
+ boolean useLegacyImplementation = false;
+ String requestedVersionString =
+ input
+ .getPipeline()
+ .getOptions()
+ .as(StreamingOptions.class)
+ .getUpdateCompatibilityVersion();
+ if (requestedVersionString != null) {
+ List<String> requestedVersion =
Arrays.asList(requestedVersionString.split("\\."));
+ List<String> targetVersion = Arrays.asList("2", "59", "0");
+
+ if (Comparators.lexicographical(Comparator.<String>naturalOrder())
+ .compare(requestedVersion, targetVersion)
+ < 0) {
+ useLegacyImplementation = true;
Review Comment:
Readability: I prefer the approach of a hard fork, where the first thing you
do with expand is to take a different path depending on the version. This keeps
the cyclomatic complexity of the main logic to a minimum. It is also more
precise than "legacy" which has some connotations but isn't as good as
`expand_2_59_0`. I can see how here you are only changing a tiny bit of logic
based on this, but I would still prefer distinct methods called where each
method is a straight-line implementation for a particular version range.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]