-
Notifications
You must be signed in to change notification settings - Fork 2.2k
[Feature][Connectors-v2] kafka support multiple table sink #10154
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
misi1987107
wants to merge
16
commits into
apache:dev
Choose a base branch
from
misi1987107:feature-kafkasink-multipletable
base: dev
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from 9 commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
8e3f4b7
[feature] kafka support multiple table sink
20ffae5
[feature] Add data cleaning strategy
01dc1b1
[improve] kafka consume data adjust
8fc22e4
[improve] modify the parallelism = 1
2097aba
[improve] modify task cancellation status
28a3c24
[improve] add config MULTI_TABLE_SINK_REPLICA
344ff8d
Merge branch 'dev' into feature-kafkasink-multipletable
9c01571
[improve] modify createWriter()
32d5b57
[improve] code format
c1a44de
[improve] Merge KafkaMultiTableSinkIT into KafkaIT
22256b7
[improve] revise the document
42293f3
Merge branch 'dev' into feature-kafkasink-multipletable
198a884
Merge branch 'refs/heads/dev' into feature-kafkasink-multipletable
70058ca
Merge branch 'dev' into feature-kafkasink-multipletable
7aab83c
[improve] revert job status CANCELING to CANCELED
603d051
[improve] revert
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
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
51 changes: 51 additions & 0 deletions
51
.../org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaMultiTableResourceManager.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,51 @@ | ||
| /* | ||
| * 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.seatunnel.connectors.seatunnel.kafka.sink; | ||
|
|
||
| import org.apache.seatunnel.api.sink.MultiTableResourceManager; | ||
|
|
||
| import lombok.AllArgsConstructor; | ||
| import lombok.extern.slf4j.Slf4j; | ||
|
|
||
| import java.util.Optional; | ||
|
|
||
| /** | ||
| * Multi-table resource manager for Kafka sink. | ||
| * | ||
| * <p>Manages the shared KafkaProducerManager across multiple table writers. | ||
| */ | ||
| @AllArgsConstructor | ||
| @Slf4j | ||
| public class KafkaMultiTableResourceManager | ||
| implements MultiTableResourceManager<KafkaProducerManager> { | ||
|
|
||
| private final KafkaProducerManager producerManager; | ||
|
|
||
| @Override | ||
| public Optional<KafkaProducerManager> getSharedResource() { | ||
| return Optional.of(producerManager); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| log.info("Closing Kafka multi-table resource manager"); | ||
| if (producerManager != null) { | ||
| producerManager.close(); | ||
| } | ||
| } | ||
| } |
132 changes: 132 additions & 0 deletions
132
.../main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaProducerManager.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,132 @@ | ||
| /* | ||
| * 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.seatunnel.connectors.seatunnel.kafka.sink; | ||
|
|
||
| import org.apache.kafka.clients.producer.KafkaProducer; | ||
| import org.apache.kafka.clients.producer.ProducerConfig; | ||
|
|
||
| import lombok.Getter; | ||
| import lombok.extern.slf4j.Slf4j; | ||
|
|
||
| import java.util.Map; | ||
| import java.util.Properties; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
|
|
||
| /** | ||
| * Manages shared KafkaProducer instances for multi-table sink. | ||
| * | ||
| * <p>For non-transaction mode: All writers share a single KafkaProducer instance. | ||
| * | ||
| * <p>For exactly-once mode: Each queue index has its own KafkaProducer with unique | ||
| * transactional.id. | ||
| */ | ||
| @Slf4j | ||
| @Getter | ||
| public class KafkaProducerManager { | ||
|
|
||
| private final Properties kafkaProperties; | ||
| private final boolean isExactlyOnce; | ||
|
|
||
| /** Shared producer for non-transaction mode */ | ||
| private volatile KafkaProducer<byte[], byte[]> sharedProducer; | ||
|
|
||
| /** Transaction producers for exactly-once mode, keyed by queue index */ | ||
| private final Map<Integer, KafkaProducer<byte[], byte[]>> transactionProducers; | ||
|
|
||
| public KafkaProducerManager(Properties kafkaProperties, boolean isExactlyOnce) { | ||
| this.kafkaProperties = kafkaProperties; | ||
| this.isExactlyOnce = isExactlyOnce; | ||
| this.transactionProducers = new ConcurrentHashMap<>(); | ||
|
|
||
| if (!isExactlyOnce) { | ||
| log.info("Creating shared KafkaProducer for non-transaction mode"); | ||
| this.sharedProducer = new KafkaProducer<>(kafkaProperties); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Get a KafkaProducer instance. | ||
| * | ||
| * @param queueIndex the queue index for transaction mode | ||
| * @param transactionPrefix the transaction id prefix for exactly-once mode | ||
| * @return KafkaProducer instance | ||
| */ | ||
| public KafkaProducer<byte[], byte[]> getProducer(int queueIndex, String transactionPrefix) { | ||
| if (!isExactlyOnce) { | ||
| return sharedProducer; | ||
| } | ||
|
|
||
| return transactionProducers.computeIfAbsent( | ||
| queueIndex, | ||
| idx -> { | ||
| Properties props = new Properties(); | ||
| props.putAll(kafkaProperties); | ||
| String transactionalId = transactionPrefix + "-shared-" + idx; | ||
| props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId); | ||
| log.info( | ||
| "Creating transactional KafkaProducer for queue index {} with transactional.id: {}", | ||
| idx, | ||
| transactionalId); | ||
| KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(props); | ||
| producer.initTransactions(); | ||
| return producer; | ||
| }); | ||
| } | ||
|
|
||
| /** | ||
| * Check if the producer for the given queue index exists. | ||
| * | ||
| * @param queueIndex the queue index | ||
| * @return true if the producer exists | ||
| */ | ||
| public boolean containsProducer(int queueIndex) { | ||
| if (!isExactlyOnce) { | ||
| return sharedProducer != null; | ||
| } | ||
| return transactionProducers.containsKey(queueIndex); | ||
| } | ||
|
|
||
| /** Close all KafkaProducer instances. */ | ||
| public void close() { | ||
| if (sharedProducer != null) { | ||
| log.info("Closing shared KafkaProducer"); | ||
| try { | ||
| sharedProducer.flush(); | ||
| sharedProducer.close(); | ||
| } catch (Exception e) { | ||
| log.warn("Failed to close shared KafkaProducer", e); | ||
| } | ||
| sharedProducer = null; | ||
| } | ||
|
|
||
| transactionProducers.forEach( | ||
| (idx, producer) -> { | ||
| log.info("Closing transactional KafkaProducer for queue index {}", idx); | ||
| try { | ||
| producer.flush(); | ||
| producer.close(); | ||
| } catch (Exception e) { | ||
| log.warn( | ||
| "Failed to close transactional KafkaProducer for queue index {}", | ||
| idx, | ||
| e); | ||
| } | ||
| }); | ||
| transactionProducers.clear(); | ||
| } | ||
| } |
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.
${database_name} and ${schema_name} should also be supported
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.
The ${datasample_name} ${schema_name} variable is already supported, I will modify the document