-
Notifications
You must be signed in to change notification settings - Fork 13.3k
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
(release-1.3) [FLINK-7143] [kafka] Fix indeterminate partition assignment in FlinkKafkaConsumer #4301
Conversation
I think that would fix the bug. There are two things I would like to improve, though:
I would suggest to have a function |
@StephanEwen thanks for the review. Your suggestion makes a lot of sense. I've fixed this up as the following:
|
* @return index of the target subtask that the Kafka partition should be assigned to. | ||
*/ | ||
public static int assign(KafkaTopicPartition partition, int numParallelSubtasks) { | ||
int startIndex = Math.abs(partition.getTopic().hashCode() * 31 % numParallelSubtasks); |
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.
Minor detail: Math.abs
does not work for Integer.MIN_VALUE
, so it is slightly safer to do
int startIndex = ((partition.getTopic().hashCode() * 31) & 0x7FFFFFFF) % numParallelSubtasks);
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.
Good catch!
Quick comment for my own clarification: when restoring from a 1.3.x savepoint, the new assignment logic will not be used, right? In Flink 1.3.x there is no dynamic partition discovery and so when restarting from state we have to strictly stick to what we have in the (operator) state to avoid reading partitions on multiple subtasks. If this is true, this also means that folks that have savepoints on 1.3.x cannot them if they want to benefit from this fix, right? |
Yes, that is true. This assignment logic is only applied on fresh starts. |
This would then mean we discourage restoring from a 1.3.x savepoint, because the state is potentially incorrect. |
Do we have a test for the case where there are fewer partitions than sources so that some sources do not get partitions on restore? To make sure they do not accidentally re-discover? |
Just to double-check: I see that the state of the partitions is in a |
@StephanEwen Regarding For |
@aljoscha on some second thinking, I don't think we can easily deal with the fact that, when restoring from 1.3.1 / 1.3.0 savepoints in 1.3.2, users will not benefit from this bug fix. There is basically no guarantee in what the distribution would be when restoring from 1.3.1 / 1.3.0, and therefore no way to manipulate it to follow the new fixed distribution scheme we introduce here. |
Yes, I don't think we can get around this when restoring from "old" state. I also have another suspicion: I don't think that |
Note, that this doesn't normally occur because the strategy for assigning Kafka partitions and for assigning operator state is the same (right now). However, this means that you will have active partition discovery for parallel instances that didn't previously have state: assume we have 1 partition and 1 parallel source. Now we add a new partition and restart the Flink job. Now, parallel instance 1 will still read from partition 0, parallel instance 2 will think that it didn't restart (because it didn't get state) and will discover partitions and take ownership of partition 1. (This is with current |
Oye, this is more complicated than I thought. On This PR breaks the alignment because the flink/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java Line 561 in b1f7621
The fix is to properly forward the information of whether we're restored in |
…ate in FlinkKafkaConsumer Previously, querying the partition list and using it to filter out restored partition states is problematic since the queried partition list may be missing partitions due to temporary downtime of Kafka brokers. Effectively, this caused the potential dropping of state on restores. This commit fixes this by completely removing partition querying if we're restoring state (as notified by FunctionInitializationContext.isRestored()). The subscribed partitions will always be exactly what the restored state contains. This closes apache#4357. This closes apache#4344. This closes apache#4301.
This was merged via #4357. Closing .. |
This PR changes the mod operation for partition assignment from
i % numTasks == subtaskIndex
topartition.hashCode % numTasks == subtaskIndex
.The bug was initially caused by #3378, when moving away from sorting the partition list. Apparently, the tests for partition assignment was not strict enough and did not catch this. This PR additionally adds verifications that the partitions end up in the expected subtasks, and that different partition ordering will still have the same partition assignments.
Note: a fix is not required for the
master
branch, since the partition discovery changes already indirectly fixed the issue. However, test coverage for deterministic assignment should likewise be improved inmaster
as well. A separate PR will be opened for that.