Skip to content

[FLINK-35870] Change DEFAULT_KAFKA_TRANSACTION_TIMEOUT in KafkaSinkBuilder #110

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

Open
wants to merge 1 commit into
base: main
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
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@
public class KafkaSinkBuilder<IN> {

private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkBuilder.class);
private static final Duration DEFAULT_KAFKA_TRANSACTION_TIMEOUT = Duration.ofHours(1);
private static final Duration DEFAULT_KAFKA_TRANSACTION_TIMEOUT = Duration.ofMinutes(15);

Choose a reason for hiding this comment

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

It seems a reasonable change, but I am concerned about changing the default of an option. The cautious approach would be to introduce a new parameter with the new default, so the user can choose when to move to the new default and there are no unexpected changes in behaviour for minor upgrades.

private static final String[] warnKeys =
new String[] {
ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.kafka.clients.producer.ProducerConfig;
import org.junit.jupiter.api.Test;

import java.time.Duration;
import java.util.Arrays;
import java.util.Properties;
import java.util.function.Consumer;
Expand Down Expand Up @@ -87,6 +88,27 @@ public void testBootstrapServerSetting() {
p -> assertThat(p).containsKeys(DEFAULT_KEYS));
}

@Test
public void testTransactionTimeoutSetting() {
validateProducerConfig(
getBasicBuilder(),
p -> {
assertThat(p.get(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG))
.isEqualTo((int) Duration.ofMinutes(15).toMillis());
});

Properties testConf = new Properties();
testConf.put(
ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, (int) Duration.ofHours(1).toMillis());

validateProducerConfig(
getBasicBuilder().setKafkaProducerConfig(testConf),
p -> {
assertThat(p.get(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG))
.isEqualTo((int) Duration.ofHours(1).toMillis());
});
}

private void validateProducerConfig(
KafkaSinkBuilder<?> builder, Consumer<Properties> validator) {
validator.accept(builder.build().getKafkaProducerConfig());
Expand Down