Skip to content
New issue

Have a question about this project? # for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “#”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? # to your account

[FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader #100

Merged
merged 1 commit into from
Sep 17, 2024

Conversation

dongwoo6kim
Copy link
Contributor

@dongwoo6kim dongwoo6kim commented Apr 29, 2024

Problem

When using the flink kafka connector in batch scenarios, consuming transactional messages can cause indefinite hanging.
This issue can be easily reproduced with following steps.

  1. Produce transactional messages and commit them.
  2. Configure scan.bounded.mode to latest-offset and run consumer using flink kafka connector

Cause

The previous stopping condition in the KafkaPartitionSplitReader compared the offset of the last record with the stoppingOffset. This approach works for streaming use cases and batch processing of non-transactional messages. However, in scenarios involving transactional messages, this is insufficient.
Control messages, which are not visible to clients, can occupy the entire range between the last record's offset and the stoppingOffset which leads to indefinite blocking.

Workaround

I've modified the stopping condition to use consumer.position(tp), which effectively skips any control messages present in the current poll, pointing directly to the next record's offset.
To handle edge cases, particularly when properties.max.poll.records is set to 1, I've adjusted the fetch method to always check all assigned partitions, even if no records are returned in a poll.

Edge case example

Consider partition 0, where offsets 13 and 14 are valid records and 15 is a control record. If stoppingOffset is set to 15 for partition 0and properties.max.poll.records is configured to 1, checking only partitions that return records would miss offset 15. By consistently reviewing all assigned partitions, the consumer’s position jumps control record in the subsequent poll, allowing the system to escape.

Discussion

To address the metric issue in FLINK-33484, I think we need to make wrapper class of ConsumerRecord for example ConsumerRecordWithOffsetJump.

public ConsumerRecordWithOffsetJump(ConsumerRecord<K, V> record, long offsetJump) {
        this.record = record;
        this.offsetJump = offsetJump;
    }

And we may need new KafkaPartitionSplitReader that implements
SplitReader<ConsumerRecordWithOffsetJump<byte[], byte[]>, KafkaPartitionSplit>.
So when record is emitted it should set current offset not just record.offset()+1 but
record.offset() + record.jumpValue in here.
jumpValue is typically 1, except for the last record of each poll where it's calculated as
consumer.position() - lastRecord.offset().
If this sounds good to everyone, I'm happy to work on this.

Copy link

boring-cyborg bot commented Apr 29, 2024

Thanks for opening this pull request! Please check out our contributing guidelines. (https://flink.apache.org/contributing/how-to-contribute.html)

@LinMingQiang
Copy link

its work , i had try.

stoppingOffset);
recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset);
finishSplitAtRecord(
tp, stoppingOffset, consumerPosition, finishedPartitions, recordsBySplits);
}
// Track this partition's record lag if it never appears before
kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp);

Choose a reason for hiding this comment

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

 consumerRecords.partitions().forEach(trackTp -> {
            // Track this partition's record lag if it never appears before
            kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, trackTp);
        });

Choose a reason for hiding this comment

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

we do not need to track tp when consumerRecords is empty.

Copy link
Contributor Author

@dongwoo6kim dongwoo6kim May 15, 2024

Choose a reason for hiding this comment

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

@LinMingQiang, thanks for the review.
I've changed to track tp, only when there is record for that tp.

@dongwoo6kim dongwoo6kim force-pushed the FLINK-34470 branch 2 times, most recently from e9bcea2 to 3e07d38 Compare May 15, 2024 06:39
Copy link

@morazow morazow left a comment

Choose a reason for hiding this comment

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

Hey @dongwoo6kim, overall seems okay from my side. But would be possible to add a integration-test ITCase for this case

@dongwoo6kim
Copy link
Contributor Author

Hello @morazow I've added ITCase for this case.
This case fails for current main branch due to timeout and works okay with fixed code.

@morazow
Copy link

morazow commented Jun 10, 2024

Thanks @dongwoo6kim ,

Tests looks good from my side 👍

(Recently I faced similar issue which maybe related, when running batch mode with setting startingOffsets. The change should solve that issue. But we may create issue for it)

@dongwoo6kim
Copy link
Contributor Author

Thanks for confirming @morazow,
Please feel free to provide any additional advice before merging this fix.
It would be also helpful if you could elaborate more on the issue you mentioned and consider adding relevant test code for it.

@morazow
Copy link

morazow commented Jun 26, 2024

Hey @dongwoo6kim, we created another issue for it, the solution seems to be similar but let's discuss it again once this PR is merged.

@dongwoo6kim
Copy link
Contributor Author

Hello @morazow, I've added test code for the mentioned issue, please take a look.
Test passes with this fixed code and on the latest main branch, it timeouts due to indefinite blocking.

@morazow
Copy link

morazow commented Jul 8, 2024

Thanks @dongwoo6kim, Looks good!

Copy link
Contributor

@AHeise AHeise left a comment

Choose a reason for hiding this comment

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

@dongwoo6kim thank you very much for your contribution. I have a couple of remarks.

If you don't have time to fix it, I can also take over because we also hitting that on production.

List<ProducerRecord<String, Integer>> records =
KafkaSourceTestEnv.getRecordsForTopic(transactionalTopic);
// Prepare records for executeAndVerify method
records.removeIf(record -> record.partition() > record.value());
Copy link
Contributor

Choose a reason for hiding this comment

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

Why is this necessary?

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 is preprocessing step before using executeAndVerify method to verify test result.

This method expects records from partition P should be an integer sequence from P to NUM_RECORDS_PER_PARTITION. So I deleted records where the value is less than the partition number.

Similar approach to here

Copy link
Contributor

Choose a reason for hiding this comment

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

Could you change the comment to reflect how the data looks afterwards? I'm assuming this will preserve one record per partition?
I'm also curios why we need a specific data layout for this test to work? Or rephrased: what would happen if we retain all records originally generated? Wouldn't the test still assert similar things?

Copy link
Contributor Author

@dongwoo6kim dongwoo6kim Sep 17, 2024

Choose a reason for hiding this comment

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

I've made two changes.

  1. Replaced records.removeIf(record -> record.partition() > record.value()) with KafkaSourceTestEnv.setupEarliestOffsets(transactionalTopic), as it serves a similar purpose.
  2. Added a comment explaining how the data looks after the setup.

I'm assuming this will preserve one record per partition?

After data modification, each partition p will contain records from p to NUM_RECORDS_PER_PARTITION (which is 10). For example, partition 1 has records 1 to 10, and partition 5 has records 5 to 10.

what would happen if we retain all records originally generated

If we retain all records we need to make new assertion logic for the generated records.
The main purpose of this data modification setup is to reuse executeAndVerify method.
When you look at here executeAndVerify method expects the input data to be modified like this way.

I intended to reuse existing test util functions and follow the test code convention but if you think it is causing unnecessary confusion I can change the test code to have custom assertion logic.

Copy link
Contributor

Choose a reason for hiding this comment

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

If it's needed for verification, then all good. We should probably revise the setup then at a later point.

@dongwoo6kim
Copy link
Contributor Author

dongwoo6kim commented Sep 13, 2024

@AHeise thanks for the feedback!
I've addressed your comments and applied the suggested changes. When you have a moment, please take a look. Thanks

Copy link
Contributor

@AHeise AHeise left a comment

Choose a reason for hiding this comment

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

Thank you very much for the swift response and action. Changes look good to me. I will approve and merge once CI passes.

My only concern is that I don't fully understand why we need to modify the data in the test case. Shouldn't a transaction marker always be at the end? What did I miss?

Also if possible could you squash your commits and extend the commit message to include some of the information of the PR description? Basically state the problem and the briefly summarize the solution. Your commit message and PR title is already very descriptive, so it's not necessary for merging and it would just be a nice-to-have.

List<ProducerRecord<String, Integer>> records =
KafkaSourceTestEnv.getRecordsForTopic(transactionalTopic);
// Prepare records for executeAndVerify method
records.removeIf(record -> record.partition() > record.value());
Copy link
Contributor

Choose a reason for hiding this comment

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

Could you change the comment to reflect how the data looks afterwards? I'm assuming this will preserve one record per partition?
I'm also curios why we need a specific data layout for this test to work? Or rephrased: what would happen if we retain all records originally generated? Wouldn't the test still assert similar things?

@AHeise
Copy link
Contributor

AHeise commented Sep 16, 2024

Seems like the deleted arch unit rule was still needed. What was your intent when you deleted it?

…stopping condition on split reader

Problem: In batch mode, flink kafka connector could hang when
consuming transactional messages or reading from deleted records.

Solution: Use consumer.position() instead of lastRecord's offset to skip
control and deleted messages, preventing the hang.
@dongwoo6kim
Copy link
Contributor Author

Seems like the deleted arch unit rule was still needed. What was your intent when you deleted it?

It was automatically deleted after running mvn clean verify locally. I manually rolled back the archunit changes.

@dongwoo6kim
Copy link
Contributor Author

@AHeise Thanks for the feedback. I've left some comments and made updates. Please have a look.

@AHeise
Copy link
Contributor

AHeise commented Sep 17, 2024

Changes all LGTM. I'm running CI and merge when it's green.
Thank you very much for this much needed fix!

@AHeise AHeise merged commit 122a743 into apache:main Sep 17, 2024
13 checks passed
Copy link

boring-cyborg bot commented Sep 17, 2024

Awesome work, congrats on your first merged pull request!

# for free to join this conversation on GitHub. Already have an account? # to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants