-
Notifications
You must be signed in to change notification settings - Fork 136
Fix KoP will cause Kafka Errors.REQUEST_TIMED_OUT when consume multi TopicPartition in one consumer request #654
Fix KoP will cause Kafka Errors.REQUEST_TIMED_OUT when consume multi TopicPartition in one consumer request #654
Conversation
…Context.handleFetch(), tryComplete() may enter race condition When one topicPartition removed from responseData, maybe removed again, which will cause Errors.REQUEST_TIMED_OUT
@BewareMyPower Hi, Yunze brother, how about your think of this issue? PTAL. :) |
tests/src/test/java/io/streamnative/pulsar/handlers/kop/MessageFetchContextTest.java
Outdated
Show resolved
Hide resolved
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.
Great find. Just leave some suggestions.
- We use TestNG in KoP, please replace JUnit with TestNG.
MessageFetchContextTest
doesn't rely on theKopProtocolHandlerTestBase
, so it should be put underkafka-impl/tests
instead of thetests/
directory.MessageFetchContextTest
doesn't testMessageFetchContext
directly, it looks like a simulation. i.e, ifMessageFetchContext
was modified in future, this test would still pass so it couldn't protect the code.
@BewareMyPower Thanks for your review very much. I will modify it to TestNG and move it to the correct directory, I will consider modifying the test cases to protect the code. Thanks. :) |
@BewareMyPower Hi, Yunze brother, I have addressed your comment. PTAL :) |
It looks like your test cannot pass, see
|
If it's not convenient to add the test, it will be acceptable to not add the test. The race condition happens when |
@BewareMyPower Therefore, we need to make a null pointer judgment on responseData in tryComplete. |
…micBoolean to avoid thread blocking & fix test failed
You're right, I think
in |
kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/MessageFetchContext.java
Outdated
Show resolved
Hide resolved
@BewareMyPower for Codacy Static Code Analysis error, what should I do, please teach me. :) |
@BewareMyPower All checks have passed, PTAL :) |
The test can be simplified. IMO, only |
@BewareMyPower Gotcha! I have addressed your comment. PTAL |
I still have questions about the test. When I modified the private void tryComplete() {
if (responseData.size() >= fetchRequest.fetchData().size()) {
complete();
}
} The test can still pass in my local env. |
@BewareMyPower I set invocationCount = 5000 locally and copied REQUEST_TIMED_OUT once and this.responseData is empty three times. I didn’t set invocationCount = 5000 in pr because I think if the code has concurrency problems, it will definitely become a flaky test in future code submissions. , What do you think should be done? |
Okay, it makes sense. |
…TopicPartition in one consumer request (#654) Fixes #604 ### Motivation When consume multi TopicPartition in one request, in MessageFetchContext.handleFetch(), tryComplete() may enter race condition When one topicPartition removed from responseData, maybe removed again, which will cause Errors.REQUEST_TIMED_OUT ReadEntries and CompletableFuture.complete operations for each partition are all performed by BookKeeperClientWorker- Different threads in the OrderedExecutor thread pool are executed. When the partition can read data, because the read data and decode operations will take uncertain time, the competition in this case is relatively weak; and when the partition has no data to write, and the consumer After all the data has been consumed, I have been making empty fetch requests, which can be reproduced stably at this time. Stable steps to reproduce: A single broker has two partition leaders for one topic; The topic is not writing data, and consumers have consumed the old data; At this time, the consumer client continues to send Fetch requests to broker; Basically, you will soon see that the server returns error_code=7, and the client will down。 1、One fetch request, two partitions, and two threads. The data obtained is an empty set without any protocol conversion operation. 2、When the BookKeeperClientWorker-OrderedExecutor-25-0 thread adds test_kop_222-1 to the responseData, BookKeeperClientWorker-OrderedExecutor-23- 0 thread adds test_kop_222-3 to responseData, 3、at this time responseData.size() >= fetchRequest.fetchData().size(), because tryComplete has no synchronization operation, two threads enter at the same time, 4、fetchRequest.fetchData().keySet() .forEach two threads traverse at the same time, resulting in the same partition multiple times responseData.remove(topicPartition), partitionData is null and cause the REQUEST_TIMED_OUT error. ![image](https://user-images.githubusercontent.com/35599757/129462871-37fbfc6f-1603-4da8-9815-95a278195936.png) ### Modifications `MessageFetchContext.tryComplete` add synchronization lock
Fixes #604
Motivation
When consume multi TopicPartition in one request, in MessageFetchContext.handleFetch(), tryComplete() may enter race condition
When one topicPartition removed from responseData, maybe removed again, which will cause Errors.REQUEST_TIMED_OUT
ReadEntries and CompletableFuture.complete operations for each partition are all performed by BookKeeperClientWorker- Different threads in the OrderedExecutor thread pool are executed. When the partition can read data, because the read data and decode operations will take uncertain time, the competition in this case is relatively weak; and when the partition has no data to write, and the consumer After all the data has been consumed, I have been making empty fetch requests, which can be reproduced stably at this time.
Stable steps to reproduce:
A single broker has two partition leaders for one topic;
The topic is not writing data, and consumers have consumed the old data;
At this time, the consumer client continues to send Fetch requests to broker;
Basically, you will soon see that the server returns error_code=7, and the client will down。
1、One fetch request, two partitions, and two threads. The data obtained is an empty set without any protocol conversion operation.
2、When the BookKeeperClientWorker-OrderedExecutor-25-0 thread adds test_kop_222-1 to the responseData, BookKeeperClientWorker-OrderedExecutor-23- 0 thread adds test_kop_222-3 to responseData,
3、at this time responseData.size() >= fetchRequest.fetchData().size(), because tryComplete has no synchronization operation, two threads enter at the same time,
4、fetchRequest.fetchData().keySet() .forEach two threads traverse at the same time, resulting in the same partition multiple times responseData.remove(topicPartition), partitionData is null and cause the REQUEST_TIMED_OUT error.
Modifications
MessageFetchContext.tryComplete
add synchronization lock