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
KAFKA-8334 Make sure the thread which tries to complete delayed reque… #8657
Merged
Merged
Changes from all commits
Commits
Show all changes
17 commits
Select commit
Hold shift + click to select a range
c086bd2
KAFKA-8334 Make sure the thread which tries to complete delayed reque…
chia7712 c8f54ad
address review comment
chia7712 dfb9b4b
add more comment
chia7712 bb7a9e5
address review comments
chia7712 3e89824
address review comments
chia7712 9c981c5
revert leaderHWIncremented to option
chia7712 eba6df8
introduce leaderHWChange and consume all delayed actions
chia7712 4a9e49f
revise comment
chia7712 c42f464
revert action queue in per server
chia7712 e1a6044
Incremental -> Increased; LeaderHWChange -> LeaderHwChange; and other…
chia7712 5da2fab
fix deadlock in TransactionCoordinatorConcurrencyTest
chia7712 4e66db0
fix potential deadlock in tryCompleteElseWatch
chia7712 b3cd7ad
a bit tweak
chia7712 50032c1
remove unused methods from DelayedOperations; add safeTryCompleteAndE…
chia7712 28b6854
rename safeTryCompleteOrElse to safeTryCompleteAndElse; revise comment
chia7712 9a49f04
remove global variable from GroupCoordinatorConcurrencyTest; revise c…
chia7712 fbd4656
tweak comment
chia7712 File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains 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 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 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 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 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,56 @@ | ||
/** | ||
* 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 kafka.server | ||
|
||
import java.util.concurrent.ConcurrentLinkedQueue | ||
|
||
import kafka.utils.Logging | ||
|
||
/** | ||
* This queue is used to collect actions which need to be executed later. One use case is that ReplicaManager#appendRecords | ||
* produces record changes so we need to check and complete delayed requests. In order to avoid conflicting locking, | ||
* we add those actions to this queue and then complete them at the end of KafkaApis.handle() or DelayedJoin.onExpiration. | ||
*/ | ||
class ActionQueue extends Logging { | ||
private val queue = new ConcurrentLinkedQueue[() => Unit]() | ||
|
||
/** | ||
* add action to this queue. | ||
* @param action action | ||
*/ | ||
def add(action: () => Unit): Unit = queue.add(action) | ||
|
||
/** | ||
* try to complete all delayed actions | ||
*/ | ||
def tryCompleteActions(): Unit = { | ||
val maxToComplete = queue.size() | ||
var count = 0 | ||
var done = false | ||
while (!done && count < maxToComplete) { | ||
try { | ||
val action = queue.poll() | ||
if (action == null) done = true | ||
else action() | ||
} catch { | ||
case e: Throwable => | ||
error("failed to complete delayed actions", e) | ||
} finally count += 1 | ||
} | ||
} | ||
} |
This file contains 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 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 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
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.
With this change,
DelayedOperations.checkAndCompleteFetch()
is only used in tests. I am wondering if it can be removed. It's fine if we want to do this in a followup PR.Unrelated to this PR,
DelayedOperations.checkAndCompleteProduce
andDelayedOperations.checkAndCompleteDeleteRecords
seem unused. We can probably remove them in a separate PR.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.
I will file a ticket after this PR is merged.
this is small change. I will remove them in this PR