Skip to content
Open
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -118,36 +118,43 @@ static StreamingCommitFinalizer create(BoundedQueueExecutor workExecutor) {
* has been successfully committed to the backing state store.
*/
public void cacheCommitFinalizers(Map<Long, Pair<Instant, Runnable>> callbacks) {
List<FinalizationInfo> finalizeInfos = new ArrayList<>();
for (Map.Entry<Long, Pair<Instant, Runnable>> entry : callbacks.entrySet()) {
Long finalizeId = entry.getKey();
final FinalizationInfo info =
finalizeInfos.add(
FinalizationInfo.create(
finalizeId, entry.getValue().getLeft(), entry.getValue().getRight());

entry.getKey(), entry.getValue().getLeft(), entry.getValue().getRight()));
}
if (!finalizeInfos.isEmpty()) {
boolean shouldStartCleanupThread = false;
lock.lock();
try {
FinalizationInfo existingInfo = commitFinalizationCallbacks.put(finalizeId, info);
if (existingInfo != null) {
throw new IllegalStateException(
"Expected to not have any past callbacks for bundle "
+ finalizeId
+ " but had "
+ existingInfo);
for (FinalizationInfo info : finalizeInfos) {
FinalizationInfo existingInfo = commitFinalizationCallbacks.put(info.getId(), info);
if (existingInfo != null) {
throw new IllegalStateException(
"Expected to not have any past callbacks for bundle "
+ info.getId()
+ " but had "
+ existingInfo);
}
cleanUpQueue.add(info);
@SuppressWarnings("ReferenceEquality")
boolean newMin = cleanUpQueue.peek() == info;
if (newMin) {
queueMinChanged.signal();
}
}
if (!cleanUpThreadStarted) {
// Start the cleanup thread lazily for pipelines that don't use finalization callbacks
// and some tests.
// and some tests. Run the thread without the lock held.
cleanUpThreadStarted = true;
finalizationExecutor.execute(this::cleanupThreadBody, 0);
}
cleanUpQueue.add(info);
@SuppressWarnings("ReferenceEquality")
boolean newMin = cleanUpQueue.peek() == info;
if (newMin) {
queueMinChanged.signal();
shouldStartCleanupThread = true;
}
} finally {
lock.unlock();
if (shouldStartCleanupThread) {
finalizationExecutor.forceExecute(this::cleanupThreadBody, 0);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

cleanupThreadBody won't ever terminate unless the worker is shutdown right? If so, running cleanupThreadBody in finalizationExecutor will take away a harness thread. Worker will deadlock when run with harnessThreads set to 1. Can we create a separate cleanupThread/threadpool?

We could use a ScheduledThreadPoolExecutor + the schedule methods and remove the custom expiry logic.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I think this is a great idea!

}
}
}
}
Expand Down
Loading