Skip to content

Conversation

@kumarpritam863
Copy link
Contributor

Few Observations:

  • In the ICR mode "open" receives only the "newly added partitions", and "open" will "not be called" by connect framework if there are No New Partitions Assigned to the Task.

  • Similarly in case of "Close" the Task receives only the "removed partitions" but we blindly close the "Co-ordinator".

  • The coordinator is created only in case "open" is called but in case when a partition is revoked and no partition is added on the task then only close will be called with that revoked partition without any open call.

How this is leading to NO-Coordinator Scenario:

  • Consider the case when a partition other than partition ZERO is removed from the leader task and is assigned to some other task.
  • In this case a close call on leader will close the Co-ordinator but since this task will not get open call this will not lead to leader-election on this task.
  • As the other task which received the removed task has not received partition zero, leader election on that task will also not lead to that task being elected as leader.

Let's see this with the below example:

Initially we had one worker "W0" with two tasks "T0" and "T1" consuming from two partitions of one topic namely "P0" and "P1", so the initial configuration is:

W0 -> [{T0,P0}, {T1, P1}] -> this will elect "T0" as the co-ordinator as it has "P0"

Now another worker "W1" joins:-> this will lead to rebalancing on both tasks as well as topic partitions within those tasks.

  • Connect Framework will stop T1 on W0.
  • This will cause partition level rebalance and partition P1 will be assigned to T0.

State at this point of time:

W0 -> [{T0,[P0, P1]}]
W1 -> []

Now,

  • Connect FrameWork will start the T1 on W1.
  • This will again lead to a rebalance at the partition level.

Assume P1 is removed from T0:

  • Connect Framework will call "close" call on T0 with Partition P1.
  • This will close the Co-ordinator on T0.
  • An "open" call we be made by the Connect FrameWork on T1 with partition P1.
  • This will lead to leader-election on T1 but since it is assigned P1, T1 will not be elected as Co-ordinator.

Hence this leads to a No-Coordinator scenario.

Data Loss Scenario:

In Incremental Cooperative Rebalancing (ICR) mode, when rebalance happens, consumers do not stop consuming as their is no stop the world like in "Eager" mode of rebalancing.
In case a partition is removed from a task, Consumer co-ordinator calls "close(Collection()) of the sink Task. In this call since we are blindly dumping all the files, this will dump the records also for the partitions still retained by this task. Moreover close call will make the committer null and since we have not null check for commiter in the put(Collection) , this will silently ignore the records. Once we get an open(Collection) call from Kafka this will start the commiter without resetting the offsets which leads to data loss.

Document explaining both the scenario: https://docs.google.com/document/d/1okqGq1HXu2rDnq88wIlVDv0EmNFZYB1PhgwyAzWIKT8/edit?tab=t.0#heading=h.51qcys2ewbsa

coordinatorThread.terminate();
coordinatorThread = null;
@Override
public void stop(ResourceType resourceType) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Likewise, we can just call the type-specific methods directly instead.

@bryanck
Copy link
Contributor

bryanck commented Feb 23, 2025

We should add some comments, and also tests if feasible. Also looks like there are some code formatting issues.

@bryanck bryanck changed the title Handling no coordinator and data loss in current design in icr mode Handling no coordinator and data loss in ICR mode Feb 24, 2025
committer = CommitterFactory.createCommitter(config);
committer.start(catalog, config, context);
// We should be starting co-ordinator only the list of partitions has the zeroth partition.
if(committer.isCoordinator(partitions)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

We should move this logic into the committer implementation.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If we want to move this logic to the committer then we need to somehow pass the the partitions information and that would require modification to the commuter method signatures

…_data_loss_in_current_design_in_ICR_mode_test_latest

Handling no coordinator and data loss in current design in icr mode test latest
@bryanck bryanck changed the title Handling no coordinator and data loss in ICR mode Kafka Connect: Handling no coordinator and data loss in ICR mode Mar 13, 2025

class CommitterFactory {
static Committer createCommitter(IcebergSinkConfig config) {
static Committer createCommitter() {
Copy link
Contributor

Choose a reason for hiding this comment

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

We should leave this as-is for now and address changes in the PR to support different committers.

@Override
public void start(Map<String, String> props) {
this.config = new IcebergSinkConfig(props);
// Catalog and committer are global resources and do not depend on the topic partition;
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't feel this comment is necessary

private Collection<MemberDescription> membersWhenWorkerIsCoordinator;
private final AtomicBoolean isCommitterInitialized = new AtomicBoolean(false);

void initializeCommitter(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) {
Copy link
Contributor

Choose a reason for hiding this comment

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

This should be private.

private final AtomicBoolean isCommitterInitialized = new AtomicBoolean(false);

void initializeCommitter(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) {
if (isCommitterInitialized.compareAndSet(false, true)) {
Copy link
Contributor

@bryanck bryanck Mar 13, 2025

Choose a reason for hiding this comment

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

I feel checking for initialization is being overly conservative, given this would only happen in cases where Kafka Connect isn't following the sink task contract. We can add a precondition check instead, in methods that require initialization. Also, we shouldn't need an atomic as methods should be called from the main task thread.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This check is kind of required as the objects being created here are passed as reference to the worker and coordinator and every open call is changing that especially the KafkaClient. Also this is not a lock and this will be a synchronous call for all the task. This also prevents redundant assignment at every open call.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Also the atomic ds are very efficient and this is just a check and set and will happen only once in the lifetime of a task. Also all task are independent of each other so this will not block anything as this is not a lock just a compare and set.

Copy link
Contributor

Choose a reason for hiding this comment

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

I see, it might be a bit confusing calling committer start from the task open. What do you think of naming the new committer methods open and close instead, to align with the task API?

Copy link
Contributor

Choose a reason for hiding this comment

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

(BTW the atomic is fine, that's safer anyway, thanks)

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, I was thinking that initially but did not made change. Should I rename the new methods as open and close for committer interface.

private SinkTaskContext context;
private KafkaClientFactory clientFactory;
private Collection<MemberDescription> membersWhenWorkerIsCoordinator;
private final AtomicBoolean isCommitterInitialized = new AtomicBoolean(false);
Copy link
Contributor

Choose a reason for hiding this comment

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

We can name this initialized or isInitialized.

private Collection<MemberDescription> membersWhenWorkerIsCoordinator;
private final AtomicBoolean isCommitterInitialized = new AtomicBoolean(false);

private void initializeCommitter(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's name this just initialize

public void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) {
KafkaClientFactory clientFactory = new KafkaClientFactory(config.kafkaProps());

public boolean hasLeaderPartition(Collection<TopicPartition> currentAssignedPartitions) {
Copy link
Contributor

Choose a reason for hiding this comment

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

This should be private

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, nitpick, I feel we should remove isLeader and fold the logic in here, having the two methods with similar names is somewhat confusing.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I left that just for better readability of the code. I can make that change.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Also that method is being used in one of the tests, and did not wanted to make extra changes.

}
}

public void startWorker() {
Copy link
Contributor

Choose a reason for hiding this comment

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

The following new methods should be private

public void start(Map<String, String> props) {
this.config = new IcebergSinkConfig(props);
catalog = CatalogUtils.loadCatalog(config);
committer = CommitterFactory.createCommitter(config);
Copy link
Contributor

Choose a reason for hiding this comment

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

When assigning member variables, you should use the this. prefix.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ohh, sure, missed this.

if (isInitialized.compareAndSet(false, true)) {
this.icebergCatalog = catalog;
this.icebergSinkConfig = config;
this.sinkTaskContext = context;
Copy link
Contributor

Choose a reason for hiding this comment

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

We should revert the variable names to what they were.

}

private void startCoordinator() {
LOG.info("Task elected leader, starting commit coordinator");
Copy link
Contributor

Choose a reason for hiding this comment

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

We should protect against multiple coordinator threads here

@bryanck
Copy link
Contributor

bryanck commented Mar 14, 2025

Thanks @kumarpritam863 for the research on this and the contribution!

@bryanck bryanck merged commit 51abab1 into apache:main Mar 14, 2025
14 checks passed
@bryanck bryanck changed the title Kafka Connect: Handling no coordinator and data loss in ICR mode Kafka Connect: Handle no coordinator and data loss in ICR mode Mar 14, 2025
@kumarpritam863
Copy link
Contributor Author

Thanks @bryanck for all the support, insights and reviews.

@mblesak
Copy link

mblesak commented Jul 22, 2025

@kumarpritam863, can you please check: #13593
It is issue related to the changes in this Pull Request.
Method CommitterImpl.stop() has been marked as deprecated but I guess we need to distinguish between close() and stop() where IcebergSinkTask closes the assets like Catalog.

@kumarpritam863
Copy link
Contributor Author

@mblesak this is already handled in this PR.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants