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

Fix Kafka with Redistribute #32344

Merged
merged 5 commits into from
Sep 17, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -890,7 +890,6 @@ static <K, V> void setupExternalBuilder(
builder.setRedistributeNumKeys(0);
builder.setAllowDuplicates(false);
}
System.out.println("xxx builder service" + builder.toString());
}

private static <T> Coder<T> resolveCoder(Class<Deserializer<T>> deserializer) {
Expand Down Expand Up @@ -1697,11 +1696,17 @@ public PCollection<KafkaRecord<K, V>> expand(PBegin input) {
}

if (kafkaRead.isRedistributed()) {
// fail here instead.
checkArgument(
kafkaRead.isCommitOffsetsInFinalizeEnabled(),
"commitOffsetsInFinalize() can't be enabled with isRedistributed");
if (kafkaRead.isCommitOffsetsInFinalizeEnabled()) {
LOG.warn(
"commitOffsetsInFinalize() will not capture all work processed if set with withRedistribute()");
Naireen marked this conversation as resolved.
Show resolved Hide resolved
}
if (Boolean.TRUE.equals(
Copy link
Contributor

Choose a reason for hiding this comment

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

haven't seen this before, is this to handle null as a one-liner?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, it occurs a few more times in this file as well.

kafkaRead.getConsumerConfig().get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG))) {
LOG.warn(
"config.ENABLE_AUTO_COMMIT_CONFIG doesn't need to be set with withRedistribute()");
Copy link
Contributor

Choose a reason for hiding this comment

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

It doesn't need to but it seems like it could still be desirable if they are using the offsets for monitoring.

Should we just remove this log? It seems the correctness is the same as if we were shuffling persistently though the window is perhaps larger if a drain is being performed.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ack, removed it.

}
PCollection<KafkaRecord<K, V>> output = input.getPipeline().apply(transform);

if (kafkaRead.getRedistributeNumKeys() == 0) {
return output.apply(
"Insert Redistribute",
Expand Down Expand Up @@ -2654,6 +2659,13 @@ public PCollection<KafkaRecord<K, V>> expand(PCollection<KafkaSourceDescriptor>
if (getRedistributeNumKeys() == 0) {
LOG.warn("This will create a key per record, which is sub-optimal for most use cases.");
}
// is another check here needed for with commit offsets
if (isCommitOffsetEnabled() || configuredKafkaCommit()) {
scwhittle marked this conversation as resolved.
Show resolved Hide resolved
LOG.warn(
"Either auto_commit is set, or commitOffsetEnabled is enabled (or both), but since "
+ "withRestribute() is enabled, the runner may have additional work processed that "
+ "is ahead of the current checkpoint");
}
}

if (getConsumerConfig().get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG) == null) {
Expand Down Expand Up @@ -2687,8 +2699,7 @@ public PCollection<KafkaRecord<K, V>> expand(PCollection<KafkaSourceDescriptor>
.getSchemaCoder(KafkaSourceDescriptor.class),
recordCoder));

boolean applyCommitOffsets =
isCommitOffsetEnabled() && !configuredKafkaCommit() && !isRedistribute();
boolean applyCommitOffsets = isCommitOffsetEnabled() && !configuredKafkaCommit();
if (!applyCommitOffsets) {
return outputWithDescriptor
.apply(MapElements.into(new TypeDescriptor<KafkaRecord<K, V>>() {}).via(KV::getValue))
Expand All @@ -2710,6 +2721,15 @@ public PCollection<KafkaRecord<K, V>> expand(PCollection<KafkaSourceDescriptor>
if (Comparators.lexicographical(Comparator.<String>naturalOrder())
.compare(requestedVersion, targetVersion)
< 0) {
// Redistribute is not allowed with commits prior to 2.59.0, since there is a Reshuffle
// prior to the redistribute. The reshuffle will occur before commits are offsetted and
// before outputting KafkaRecords. Adding a redistrube then afterwards doesn't provide
// additional performance benefit.
checkArgument(
!isRedistribute(),
"Can not enable isRedistribute() while committing offsets prior to "
+ String.join(".", targetVersion));

return expand259Commits(
outputWithDescriptor, recordCoder, input.getPipeline().getSchemaRegistry());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@
import org.apache.beam.sdk.metrics.SinkMetrics;
import org.apache.beam.sdk.metrics.SourceMetrics;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.StreamingOptions;
import org.apache.beam.sdk.testing.ExpectedLogs;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
Expand Down Expand Up @@ -628,31 +629,84 @@ public void testRiskyConfigurationWarnsProperly() {
}

@Test
public void testCommitOffsetsInFinalizeAndRedistributeErrors() {
thrown.expect(Exception.class);
thrown.expectMessage("commitOffsetsInFinalize() can't be enabled with isRedistributed");

public void testCommitOffsetsInFinalizeAndRedistributeWarnings() {
int numElements = 1000;

PCollection<Long> input =
p.apply(
mkKafkaReadTransform(numElements, numElements, new ValueAsTimestampFn(), true, 0)
.commitOffsetsInFinalize()
.withConsumerConfigUpdates(
ImmutableMap.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true))
ImmutableMap.of(ConsumerConfig.GROUP_ID_CONFIG, "group_id"))
.withoutMetadata())
.apply(Values.create());

addCountingAsserts(input, numElements);
p.run();

kafkaIOExpectedLogs.verifyWarn(
"commitOffsetsInFinalize() will not capture all work processed if set with withRedistribute()");
}

@Test
public void testNumKeysIgnoredWithRedistributeNotEnabled() {
thrown.expect(Exception.class);
thrown.expectMessage(
"withRedistributeNumKeys is ignored if withRedistribute() is not enabled on the transform");

int numElements = 1000;

PCollection<Long> input =
p.apply(
mkKafkaReadTransform(numElements, numElements, new ValueAsTimestampFn(), false, 0)
.withRedistributeNumKeys(100)
.commitOffsetsInFinalize()
.withConsumerConfigUpdates(
ImmutableMap.of(ConsumerConfig.GROUP_ID_CONFIG, "group_id"))
.withoutMetadata())
.apply(Values.create());

addCountingAsserts(input, numElements);

p.run();
}

@Test
public void testDisableRedistributeKafkaOffsetLegacy() {
thrown.expect(Exception.class);
thrown.expectMessage(
"Can not enable isRedistribute() while committing offsets prior to 2.60.0");
p.getOptions().as(StreamingOptions.class).setUpdateCompatibilityVersion("2.59.0");

p.apply(
Create.of(
KafkaSourceDescriptor.of(
new TopicPartition("topic", 1),
null,
null,
null,
null,
ImmutableList.of("8.8.8.8:9092"))))
.apply(
KafkaIO.<Long, Long>readSourceDescriptors()
.withKeyDeserializer(LongDeserializer.class)
.withValueDeserializer(LongDeserializer.class)
.withRedistribute()
.withProcessingTime()
.commitOffsets());
p.run();
}

@Test
public void testEnableAutoCommitWithRedistribute() throws Exception {

int numElements = 1000;

PCollection<Long> input =
p.apply(
mkKafkaReadTransform(numElements, numElements, new ValueAsTimestampFn(), true, 0)
.withRedistribute()
.withRedistributeNumKeys(100)
.withConsumerConfigUpdates(
ImmutableMap.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true))
.withoutMetadata())
Expand All @@ -661,6 +715,9 @@ public void testNumKeysIgnoredWithRedistributeNotEnabled() {
addCountingAsserts(input, numElements);

p.run();

kafkaIOExpectedLogs.verifyWarn(
"config.ENABLE_AUTO_COMMIT_CONFIG doesn't need to be set with withRedistribute()");
}

@Test
Expand Down
Loading