-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Add kafka read override to Dataflow java runner. #36849
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
Merged
Merged
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
8cdbaf7
Add kafka read override to Dataflow java runner.
tomstepp cf76c1c
Fix spot bugs (spacing)
tomstepp 346158f
Add unit test of redistribute override
tomstepp 6335075
Update test dependencies via gradle
tomstepp 1cdd6bf
Add logic and test case for explicitly disabled.
tomstepp 1a52da3
Add explicitly enabled test case
tomstepp 67e3ca8
Use boolean asserts over assertThat, assert each read is visited only…
tomstepp File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
75 changes: 75 additions & 0 deletions
75
...ava/src/main/java/org/apache/beam/runners/dataflow/KafkaReadWithRedistributeOverride.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,75 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.beam.runners.dataflow; | ||
|
|
||
| import java.util.Map; | ||
| import org.apache.beam.sdk.io.kafka.KafkaIO; | ||
| import org.apache.beam.sdk.io.kafka.KafkaRecord; | ||
| import org.apache.beam.sdk.runners.AppliedPTransform; | ||
| import org.apache.beam.sdk.runners.PTransformMatcher; | ||
| import org.apache.beam.sdk.runners.PTransformOverrideFactory; | ||
| import org.apache.beam.sdk.util.construction.ReplacementOutputs; | ||
| import org.apache.beam.sdk.values.PBegin; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.sdk.values.TupleTag; | ||
|
|
||
| public final class KafkaReadWithRedistributeOverride { | ||
|
|
||
| private KafkaReadWithRedistributeOverride() {} | ||
|
|
||
| public static PTransformMatcher matcher() { | ||
| return new PTransformMatcher() { | ||
| @SuppressWarnings({ | ||
| "PatternMatchingInstanceof" // For compiling on older Java versions. | ||
| }) | ||
| @Override | ||
| public boolean matches(AppliedPTransform<?, ?, ?> application) { | ||
| if (application.getTransform() instanceof KafkaIO.Read) { | ||
| return ((KafkaIO.Read) application.getTransform()).isRedistributed(); | ||
| } | ||
| return false; | ||
| } | ||
| }; | ||
| } | ||
|
|
||
| /** | ||
| * {@link PTransformOverrideFactory} for {@link KafkaIO.Read} that enables {@code | ||
| * withOffsetDeduplication} when {@code withRedistribute} is enabled. | ||
| */ | ||
| static class Factory<K, V> | ||
| implements PTransformOverrideFactory< | ||
| PBegin, PCollection<KafkaRecord<K, V>>, KafkaIO.Read<K, V>> { | ||
|
|
||
| @Override | ||
| public PTransformReplacement<PBegin, PCollection<KafkaRecord<K, V>>> getReplacementTransform( | ||
| AppliedPTransform<PBegin, PCollection<KafkaRecord<K, V>>, KafkaIO.Read<K, V>> transform) { | ||
| KafkaIO.Read<K, V> read = transform.getTransform(); | ||
| if (read.getOffsetDeduplication() == null) { | ||
| return PTransformReplacement.of( | ||
| transform.getPipeline().begin(), read.withOffsetDeduplication(true)); | ||
| } | ||
| return PTransformReplacement.of(transform.getPipeline().begin(), read); | ||
| } | ||
|
|
||
| @Override | ||
| public Map<PCollection<?>, ReplacementOutput> mapOutputs( | ||
| Map<TupleTag<?>, PCollection<?>> outputs, PCollection<KafkaRecord<K, V>> newOutput) { | ||
| return ReplacementOutputs.singleton(outputs, newOutput); | ||
| } | ||
| } | ||
| } |
133 changes: 133 additions & 0 deletions
133
...src/test/java/org/apache/beam/runners/dataflow/KafkaReadWithRedistributeOverrideTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,133 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.beam.runners.dataflow; | ||
|
|
||
| import static org.hamcrest.MatcherAssert.assertThat; | ||
| import static org.hamcrest.Matchers.nullValue; | ||
| import static org.junit.Assert.assertFalse; | ||
| import static org.junit.Assert.assertTrue; | ||
|
|
||
| import java.io.Serializable; | ||
| import java.util.Collections; | ||
| import org.apache.beam.sdk.Pipeline; | ||
| import org.apache.beam.sdk.io.kafka.KafkaIO; | ||
| import org.apache.beam.sdk.runners.PTransformOverride; | ||
| import org.apache.beam.sdk.runners.TransformHierarchy.Node; | ||
| import org.apache.beam.sdk.testing.TestPipeline; | ||
| import org.apache.kafka.common.serialization.StringDeserializer; | ||
| import org.junit.Rule; | ||
| import org.junit.Test; | ||
| import org.junit.runner.RunWith; | ||
| import org.junit.runners.JUnit4; | ||
|
|
||
| @RunWith(JUnit4.class) | ||
| public class KafkaReadWithRedistributeOverrideTest implements Serializable { | ||
| @Rule public transient TestPipeline p = TestPipeline.create(); | ||
|
|
||
| @Test | ||
| public void testOverrideAppliedWhenRedistributeEnabled() { | ||
| p.apply( | ||
| "MatchingRead", | ||
| KafkaIO.<String, String>read() | ||
| .withBootstrapServers("localhost:9092") | ||
| .withTopic("test_match") | ||
| .withKeyDeserializer(StringDeserializer.class) | ||
| .withValueDeserializer(StringDeserializer.class) | ||
| .withRedistribute()); | ||
tomstepp marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| p.apply( | ||
| "NoRedistribute", | ||
| KafkaIO.<String, String>read() | ||
| .withBootstrapServers("localhost:9092") | ||
| .withTopic("test_no_redistribute") | ||
| .withKeyDeserializer(StringDeserializer.class) | ||
| .withValueDeserializer(StringDeserializer.class)); | ||
| p.apply( | ||
| "ExplicitlyDisable", | ||
| KafkaIO.<String, String>read() | ||
| .withBootstrapServers("localhost:9092") | ||
| .withTopic("test_disabled") | ||
| .withKeyDeserializer(StringDeserializer.class) | ||
| .withValueDeserializer(StringDeserializer.class) | ||
| .withOffsetDeduplication(false)); | ||
| p.apply( | ||
| "ExplicitlyEnable", | ||
| KafkaIO.<String, String>read() | ||
| .withBootstrapServers("localhost:9092") | ||
| .withTopic("test_enabled") | ||
| .withKeyDeserializer(StringDeserializer.class) | ||
| .withValueDeserializer(StringDeserializer.class) | ||
| .withRedistribute() | ||
| .withOffsetDeduplication(true)); | ||
|
|
||
| p.replaceAll( | ||
| Collections.singletonList( | ||
| PTransformOverride.of( | ||
| KafkaReadWithRedistributeOverride.matcher(), | ||
| new KafkaReadWithRedistributeOverride.Factory<>()))); | ||
|
|
||
| Pipeline.PipelineVisitor visitor = | ||
| new Pipeline.PipelineVisitor.Defaults() { | ||
|
|
||
| private boolean matchingVisited = false; | ||
| private boolean noRedistributeVisited = false; | ||
| private boolean explicitlyDisabledVisited = false; | ||
| private boolean explicitlyEnabledVisited = false; | ||
|
|
||
| @Override | ||
| public CompositeBehavior enterCompositeTransform(Node node) { | ||
| if (node.getTransform() instanceof KafkaIO.Read) { | ||
| KafkaIO.Read<?, ?> read = (KafkaIO.Read<?, ?>) node.getTransform(); | ||
| if (read.getTopics().contains("test_match")) { | ||
| assertTrue(read.isRedistributed()); | ||
| assertTrue(read.getOffsetDeduplication()); | ||
| assertFalse(matchingVisited); | ||
| matchingVisited = true; | ||
tomstepp marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } else if (read.getTopics().contains("test_no_redistribute")) { | ||
| assertFalse(read.isRedistributed()); | ||
| assertThat(read.getOffsetDeduplication(), nullValue()); | ||
| assertFalse(noRedistributeVisited); | ||
| noRedistributeVisited = true; | ||
| } else if (read.getTopics().contains("test_disabled")) { | ||
| assertFalse(read.isRedistributed()); | ||
| assertFalse(read.getOffsetDeduplication()); | ||
| assertFalse(explicitlyDisabledVisited); | ||
| explicitlyDisabledVisited = true; | ||
| } else if (read.getTopics().contains("test_enabled")) { | ||
| assertTrue(read.isRedistributed()); | ||
| assertTrue(read.getOffsetDeduplication()); | ||
| assertFalse(explicitlyEnabledVisited); | ||
| explicitlyEnabledVisited = true; | ||
| } | ||
| } | ||
| return CompositeBehavior.ENTER_TRANSFORM; | ||
| } | ||
|
|
||
| @Override | ||
| public void leaveCompositeTransform(Node node) { | ||
| if (node.isRootNode()) { | ||
| assertTrue("Matching transform was not visited", matchingVisited); | ||
| assertTrue("No redistribute transform was not visited", noRedistributeVisited); | ||
| assertTrue( | ||
| "Explicitly disabled transform was not visited", explicitlyDisabledVisited); | ||
| assertTrue("Explicitly enabled transform was not visited", explicitlyEnabledVisited); | ||
| } | ||
| } | ||
| }; | ||
| p.traverseTopologically(visitor); | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
ideally move Override to KafkaIO as
KafkaIO.Read.KAFKA_READ_OVERRIDEabove. This keeps reference to KafkaIO for DataflowRunner minimum (ease import)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.
Ah, sorry about that. @tomstepp can you send a follow-up PR to move this?
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.
Sorry, yeah I can update this, thanks for identifying the improvement @Abacn !
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.
#36887