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

[FLINK-28842] add client.id.prefix for kafka sink #118

Draft
wants to merge 1 commit into
base: main
Choose a base branch
from
Draft
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 @@ -28,6 +28,8 @@
import java.util.Map;
import java.util.Properties;

import static org.apache.flink.connector.kafka.sink.KafkaSink.withClientId;

/**
* Context providing information to assist constructing a {@link
* org.apache.kafka.clients.producer.ProducerRecord}.
Expand All @@ -37,6 +39,7 @@ public class DefaultKafkaSinkContext implements KafkaRecordSerializationSchema.K
private final int subtaskId;
private final int numberOfParallelInstances;
private final Properties kafkaProducerConfig;
private static final String clientIdSuffix = "-metadata-fetcher";

private final Map<String, int[]> cachedPartitions = new HashMap<>();

Expand All @@ -63,7 +66,8 @@ public int[] getPartitionsForTopic(String topic) {
}

private int[] fetchPartitionsForTopic(String topic) {
try (final Producer<?, ?> producer = new KafkaProducer<>(kafkaProducerConfig)) {
try (final Producer<?, ?> producer =
new KafkaProducer<>(withClientId(kafkaProducerConfig, clientIdSuffix))) {
// the fetched list is immutable, so we're creating a mutable copy in order to sort
// it
final List<PartitionInfo> partitionsList =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,13 +38,13 @@
import java.util.Properties;
import java.util.concurrent.Future;

import static org.apache.flink.connector.kafka.sink.KafkaSink.withClientId;
import static org.apache.flink.util.Preconditions.checkState;

/**
* A {@link KafkaProducer} that exposes private fields to allow resume producing from a given state.
*/
class FlinkKafkaInternalProducer<K, V> extends KafkaProducer<K, V> {

private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class);
private static final String TRANSACTION_MANAGER_FIELD_NAME = "transactionManager";
private static final String TRANSACTION_MANAGER_STATE_ENUM =
Expand All @@ -55,9 +55,10 @@ class FlinkKafkaInternalProducer<K, V> extends KafkaProducer<K, V> {
private volatile boolean inTransaction;
private volatile boolean hasRecordsInTransaction;
private volatile boolean closed;
private static final String clientIdSuffix = "";

public FlinkKafkaInternalProducer(Properties properties, @Nullable String transactionalId) {
super(withTransactionalId(properties, transactionalId));
super(withClientId(withTransactionalId(properties, transactionalId), clientIdSuffix));
this.transactionalId = transactionalId;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,15 @@
import org.apache.flink.connector.base.DeliveryGuarantee;
import org.apache.flink.core.io.SimpleVersionedSerializer;

import org.apache.kafka.clients.producer.ProducerConfig;

import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicInteger;

import static org.apache.flink.connector.kafka.source.KafkaSourceOptions.CLIENT_ID_PREFIX;

/**
* Flink Sink to produce data into a Kafka topic. The sink supports all delivery guarantees
Expand Down Expand Up @@ -56,6 +61,9 @@
public class KafkaSink<IN>
implements TwoPhaseCommittingStatefulSink<IN, KafkaWriterState, KafkaCommittable> {

private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
private static final String clientIdWriterSuffix = "-writer-";
private static final String clientIdCommitterSuffix = "-commiter";
private final DeliveryGuarantee deliveryGuarantee;

private final KafkaRecordSerializationSchema<IN> recordSerializer;
Expand Down Expand Up @@ -86,7 +94,8 @@ public static <IN> KafkaSinkBuilder<IN> builder() {
@Internal
@Override
public Committer<KafkaCommittable> createCommitter() throws IOException {
return new KafkaCommitter(kafkaProducerConfig);
return new KafkaCommitter(
maybeOverwriteClientIdPrefix(kafkaProducerConfig, clientIdCommitterSuffix));
}

@Internal
Expand All @@ -100,7 +109,8 @@ public SimpleVersionedSerializer<KafkaCommittable> getCommittableSerializer() {
public KafkaWriter<IN> createWriter(InitContext context) throws IOException {
return new KafkaWriter<IN>(
deliveryGuarantee,
kafkaProducerConfig,
maybeOverwriteClientIdPrefix(
kafkaProducerConfig, clientIdWriterSuffix + context.getSubtaskId()),
transactionalIdPrefix,
context,
recordSerializer,
Expand All @@ -114,7 +124,8 @@ public KafkaWriter<IN> restoreWriter(
InitContext context, Collection<KafkaWriterState> recoveredState) throws IOException {
return new KafkaWriter<>(
deliveryGuarantee,
kafkaProducerConfig,
maybeOverwriteClientIdPrefix(
kafkaProducerConfig, clientIdWriterSuffix + context.getSubtaskId()),
transactionalIdPrefix,
context,
recordSerializer,
Expand All @@ -132,4 +143,30 @@ public SimpleVersionedSerializer<KafkaWriterState> getWriterStateSerializer() {
protected Properties getKafkaProducerConfig() {
return kafkaProducerConfig;
}

private Properties maybeOverwriteClientIdPrefix(Properties kafkaProducerConfig, String suffix) {
String clientIdPrefix = kafkaProducerConfig.getProperty(CLIENT_ID_PREFIX.key());
if (clientIdPrefix == null) {
return kafkaProducerConfig;
}

Properties props = new Properties();
props.putAll(kafkaProducerConfig);
props.setProperty(CLIENT_ID_PREFIX.key(), clientIdPrefix + suffix);
return props;
}

public static Properties withClientId(Properties properties, String suffix) {
String clientIdPrefix = properties.getProperty(CLIENT_ID_PREFIX.key());
if (clientIdPrefix == null) {
return properties;
}
String clientId =
clientIdPrefix + suffix + "-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement();

Properties props = new Properties();
props.putAll(properties);
props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, clientId);
return props;
}
}