Skip to content
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

Narrow down the scope of waiting for pending tasks to per partition #191

Merged
merged 8 commits into from
Mar 27, 2023

Conversation

ta7uw
Copy link
Member

@ta7uw ta7uw commented Mar 2, 2023

Motivation:

Described in #172

Modifications:

  • Reload context per partition when reloading is requested

Result:

Copy link
Contributor

@ocadaruma ocadaruma left a comment

Choose a reason for hiding this comment

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

Thanks for submitting the PR.

This PR seems trying to observe pending task count per-partition.
However, it's not enough because as long as reloadRequested is true, Decaton pauses all processing so anyways none of partitions can make progress. (

return processingRateProp.value() == RateLimiter.PAUSED || reloadRequested.get();
)

So we need to manage reload-requested flag per-partition I think

@ta7uw
Copy link
Member Author

ta7uw commented Mar 6, 2023

Thanks for your review.
I'll take it into consideration.

@ta7uw
Copy link
Member Author

ta7uw commented Mar 6, 2023

@ocadaruma
Could you please review it again?

public void reloading(boolean reloading) {
this.reloading = reloading;
if (reloading) {
pause();
Copy link
Contributor

Choose a reason for hiding this comment

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

pausing here by PartitionContext itself doesn't work to pause fetching actually.

Decaton's fetch-pause works like:

So, if we pause here by PartitionContext itself, it will not be included in (1) (because it's already "paused").

Correct approach here may be holding per-partition reloading flag in PartitionContexts, and include them in partitionsNeedsPause

@ta7uw
Copy link
Member Author

ta7uw commented Mar 7, 2023

@ocadaruma
I fixed to hold reload request per-partition.
Could you please review it again?

@ocadaruma ocadaruma added new feature Add a new feature and removed new feature Add a new feature labels Mar 10, 2023
@ocadaruma ocadaruma self-requested a review March 10, 2023 02:57
@@ -53,6 +57,8 @@ public class PartitionContexts implements OffsetsStore, AssignmentStore, Partiti
private final Map<TopicPartition, PartitionContext> contexts;

private final AtomicBoolean reloadRequested;
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we still need this reloadRequested flag? Since we manage reloading states in reloadStates hash map

Copy link
Member Author

Choose a reason for hiding this comment

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

reloadRequested is useful for quickly determining if contexts should be reloaded without iteration.

Copy link
Contributor

Choose a reason for hiding this comment

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

Hm, IMO having duplicated info (with per-partition reload) could be an error prone when we maintain this code in the future, while iterating over partitions unlikely becomes a bottleneck (since we already doing iterating partitions for updating high watermarks, checking partition needs pause/resume) so the benefit to have reloadRequested separately is small.

List<TopicPartition> reloadableTopicPartitions = reloadStates.entrySet()
.stream()
.filter(entry -> entry.getValue()
&& contexts.get(entry.getKey()).pendingTasksCount() == 0)
Copy link
Contributor

Choose a reason for hiding this comment

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

This could cause NPE because currently reloadStates entry never removed even when partition is removed.

To fix that, we can add a code to remove reloadStates entry at the same time we modify contexts.

However, then, we have to maintain two separate Map which holds per-partition states, which is bothersome and an error-prone.

So I think we should reloadRequested flag in PartitionContext itself. WDYT

Copy link
Member Author

Choose a reason for hiding this comment

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

I got the point.
Having a reloadRequested flag in PartitionContext seems good.

@@ -214,7 +233,9 @@ public List<TopicPartition> partitionsNeedsPause() {
return contexts.values().stream()
.filter(c -> !c.revoking())
.filter(c -> !c.paused())
.filter(c -> pausingAll || shouldPartitionPaused(c.pendingTasksCount()))
.filter(c -> pausingAll
|| reloadStates.get(c.topicPartition())
Copy link
Contributor

Choose a reason for hiding this comment

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

Additionally, we also need to modify partitionNeedsResume to avoid returning partitions that reload is requested, because otherwise, partitions will be resumed immediately so new tasks will be kept feeding so pendingTasks never becomes 0.

@ocadaruma ocadaruma self-requested a review March 23, 2023 09:29
Copy link
Contributor

@ocadaruma ocadaruma left a comment

Choose a reason for hiding this comment

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

Left few comments but almost looks good.

Also would you mind adding an integration test to check changing the concurrency in the middle of processing doesn't cause problem? (referring RateLimiterTest might be helpful)

*/
@Getter
@Setter
private boolean reloadState;
Copy link
Contributor

Choose a reason for hiding this comment

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

  • reloadRequested sounds more clear?
  • Should be marked as volatile to ensure latest value (set by property-listener thread) is visible to subscription thread

@@ -52,7 +56,7 @@ public class PartitionContexts implements OffsetsStore, AssignmentStore, Partiti
private final int maxPendingRecords;
private final Map<TopicPartition, PartitionContext> contexts;

private final AtomicBoolean reloadRequested;
private final ReentrantLock lock;
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 rename to something like propertyReloadLock for understandability

if (!reloadRequested.getAndSet(true)) {
lock.lock();
try {
contexts.values().forEach(context -> context.reloadState(true));
Copy link
Contributor

Choose a reason for hiding this comment

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

To prevent contexts are modified during the iteration on this property-listener thread, we need to surround all places that maintaining contexts entry with lock. (currently only maybeHandlePropertyReload is surrounded, but there are several places modifying contexts)

Copy link
Member Author

Choose a reason for hiding this comment

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

Do you mean having another ReentrantLock for updating the contexts?

Copy link
Contributor

Choose a reason for hiding this comment

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

No, use propertyReloadLock.

Copy link
Contributor

@ocadaruma ocadaruma left a comment

Choose a reason for hiding this comment

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

Left last few nits comments!

Also could you fix this comment?

* Reloading this property will pause all assigned partitions until current pending tasks have done.
as reload is now done in per-partition

.filter(entry -> entry.getValue().reloadRequested()
&& entry.getValue().pendingTasksCount() == 0)
.map(Entry::getKey)
.collect(Collectors.toList());
Copy link
Contributor

Choose a reason for hiding this comment

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

[nits] toList is already static-imported so let's omit Collectors

Comment on lines 30 to 35
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Function;
import java.util.stream.Collectors;
Copy link
Contributor

Choose a reason for hiding this comment

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

[nits] There are several unused imports


logger.info("Start dropping partition contexts");
private void reloadContexts(Collection<TopicPartition> topicPartitions) {
logger.info("Start dropping partition context({})", topicPartitions);
removePartition(topicPartitions);
logger.info("Finished dropping partition contexts. Start recreating partition contexts");
Copy link
Contributor

Choose a reason for hiding this comment

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

[nits] This log and this line

should also include topicPartitions

Copy link
Contributor

@ocadaruma ocadaruma left a comment

Choose a reason for hiding this comment

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

LGTM. Nice!

@ocadaruma ocadaruma merged commit c95626a into line:master Mar 27, 2023
@ta7uw ta7uw deleted the issue-172 branch March 27, 2023 02:31
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Narrow the scope of waiting pending task count upon dynamic property reload
2 participants