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-35283] PoC for supporting unique Kafka producer client ids #101

Open
wants to merge 5 commits 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
@@ -0,0 +1,43 @@
/*
* 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.flink.connector.kafka.sink;

/**
* Construct a Kafka ClientId using a prefix
* and a subtask id.
*/
public class ClientIdFactory {
private static final String CLIENT_ID_DELIMITER = "-";

/**
* Construct a Kafka client id in the following format
* {@code clientIdPrefix-subtaskId}.
*
* @param clientIdPrefix prefix for the id
* @param subtaskId id of the Kafka producer subtask
* @return clientId
*/
public static String buildClientId(
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need to include checkpoint id? At any given time we have at least 2 producers open.

String clientIdPrefix,
int subtaskId
) {
return clientIdPrefix
+ CLIENT_ID_DELIMITER
+ subtaskId;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -61,15 +61,18 @@ public class KafkaSink<IN>
private final KafkaRecordSerializationSchema<IN> recordSerializer;
private final Properties kafkaProducerConfig;
private final String transactionalIdPrefix;
private final String clientIdPrefix;

KafkaSink(
DeliveryGuarantee deliveryGuarantee,
Properties kafkaProducerConfig,
String transactionalIdPrefix,
String clientIdPrefix,
KafkaRecordSerializationSchema<IN> recordSerializer) {
this.deliveryGuarantee = deliveryGuarantee;
this.kafkaProducerConfig = kafkaProducerConfig;
this.transactionalIdPrefix = transactionalIdPrefix;
this.clientIdPrefix = clientIdPrefix;
this.recordSerializer = recordSerializer;
}

Expand Down Expand Up @@ -102,6 +105,7 @@ public KafkaWriter<IN> createWriter(InitContext context) throws IOException {
deliveryGuarantee,
kafkaProducerConfig,
transactionalIdPrefix,
clientIdPrefix,
context,
recordSerializer,
context.asSerializationSchemaInitializationContext(),
Expand All @@ -116,6 +120,7 @@ public KafkaWriter<IN> restoreWriter(
deliveryGuarantee,
kafkaProducerConfig,
transactionalIdPrefix,
clientIdPrefix,
context,
recordSerializer,
context.asSerializationSchemaInitializationContext(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ public class KafkaSinkBuilder<IN> {

private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE;
private String transactionalIdPrefix = "kafka-sink";
private String clientIdPrefix = null;

private final Properties kafkaProducerConfig;
private KafkaRecordSerializationSchema<IN> recordSerializer;
Expand Down Expand Up @@ -190,6 +191,20 @@ public KafkaSinkBuilder<IN> setBootstrapServers(String bootstrapServers) {
return setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
}

/**
* Set the prefix for all KafkaProducer `client.id` values.
* This will overwrite any value set for `client.id` in the provided Kafka producer configuration.
* Instead, a value for `client.id` will be derived from the prefix provided.
* Using a prefix will create a unique Kafka `client.id` for all producers.
*
* @param clientIdPrefix Prefix to use
* @return {@link KafkaSinkBuilder}
*/
public KafkaSinkBuilder<IN> setClientIdPrefix(String clientIdPrefix) {
this.clientIdPrefix = checkNotNull(clientIdPrefix, "clientIdPrefix");
return this;
}

private void sanityCheck() {
checkNotNull(
kafkaProducerConfig.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG),
Expand All @@ -202,6 +217,25 @@ private void sanityCheck() {
checkNotNull(recordSerializer, "recordSerializer");
}

private static void overwriteConfig(Properties properties, String clientIdPrefix, int subtaskId) {
if (clientIdPrefix == null) {
return;
}
String updatedClientId = ClientIdFactory.buildClientId(clientIdPrefix, subtaskId);
overrideProperty(properties, ProducerConfig.CLIENT_ID_CONFIG, updatedClientId);
}

private static void overrideProperty(Properties properties, String key, String value) {
String userValue = properties.getProperty(key);
if (userValue != null) {
LOG.warn(
String.format(
"Property %s is provided but will be overridden from %s to %s",
key, userValue, value));
}
properties.setProperty(key, value);
}

Comment on lines +220 to +238
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this still needed? looks dead to me.

/**
* Constructs the {@link KafkaSink} with the configured properties.
*
Expand All @@ -210,6 +244,6 @@ private void sanityCheck() {
public KafkaSink<IN> build() {
sanityCheck();
return new KafkaSink<>(
deliveryGuarantee, kafkaProducerConfig, transactionalIdPrefix, recordSerializer);
deliveryGuarantee, kafkaProducerConfig, transactionalIdPrefix, clientIdPrefix, recordSerializer);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.flink.util.FlinkRuntimeException;

import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.Metric;
Expand Down Expand Up @@ -127,6 +128,7 @@ class KafkaWriter<IN>
DeliveryGuarantee deliveryGuarantee,
Properties kafkaProducerConfig,
String transactionalIdPrefix,
String clientIdPrefix,
Sink.InitContext sinkInitContext,
KafkaRecordSerializationSchema<IN> recordSerializer,
SerializationSchema.InitializationContext schemaContext,
Expand All @@ -136,6 +138,7 @@ class KafkaWriter<IN>
this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix, "transactionalIdPrefix");
this.recordSerializer = checkNotNull(recordSerializer, "recordSerializer");
checkNotNull(sinkInitContext, "sinkInitContext");
overwriteClientId(kafkaProducerConfig, clientIdPrefix, sinkInitContext.getSubtaskId());
this.deliveryCallback =
new WriterCallback(
sinkInitContext.getMailboxExecutor(),
Expand Down Expand Up @@ -299,6 +302,25 @@ void abortLingeringTransactions(
}
}

private static void overwriteClientId(Properties properties, String clientIdPrefix, int subtaskId) {
if (clientIdPrefix == null) {
return;
}
String updatedClientId = ClientIdFactory.buildClientId(clientIdPrefix, subtaskId);
overrideProperty(properties, ProducerConfig.CLIENT_ID_CONFIG, updatedClientId);
}

private static void overrideProperty(Properties properties, String key, String value) {
String userValue = properties.getProperty(key);
if (userValue != null) {
LOG.warn(
String.format(
"Property %s is provided but will be overridden from %s to %s",
key, userValue, value));
}
properties.setProperty(key, value);
}

/**
* For each checkpoint we create new {@link FlinkKafkaInternalProducer} so that new transactions
* will not clash with transactions created during previous checkpoints ({@code
Expand Down