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

[HUDI-4574] Fixed timeline based marker thread safety issue #6383

Merged
merged 2 commits into from
Aug 16, 2022
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -34,7 +34,7 @@
import org.apache.log4j.Logger;

import java.io.IOException;
import java.util.HashMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -74,7 +74,8 @@ public class MarkerHandler extends Handler {
// Parallelism for reading and deleting marker files
private final int parallelism;
// Marker directory states, {markerDirPath -> MarkerDirState instance}
private final Map<String, MarkerDirState> markerDirStateMap = new HashMap<>();
// Use ConcurrentHashMap to ensure thread safety in dispatchingExecutorService
private final Map<String, MarkerDirState> markerDirStateMap = new ConcurrentHashMap<>();
// A thread to dispatch marker creation requests to batch processing threads
private final MarkerCreationDispatchingRunnable markerCreationDispatchingRunnable;
private final Object firstCreationRequestSeenLock = new Object();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,8 +66,10 @@ public void run() {

// Only fetch pending marker creation requests that can be processed,
// i.e., that markers can be written to a underlying file
for (String markerDir : markerDirStateMap.keySet()) {
MarkerDirState markerDirState = markerDirStateMap.get(markerDir);
// markerDirStateMap is used in other thread, need to ensure thread safety
for (Map.Entry<String, MarkerDirState> entry : markerDirStateMap.entrySet()) {
String markerDir = entry.getKey();
MarkerDirState markerDirState = entry.getValue();
Copy link
Contributor

Choose a reason for hiding this comment

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

markerDirState.fetchPendingMarkerCreationRequests seems already thread safe ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

markerDirState.fetchPendingMarkerCreationRequests seems already thread safe ?

but markerDirStateMap is not thread safe. request add/delete MarkerDirState in markerDirStateMap, and MarkerCreationDispatching traversal markerDirStateMap in other thread. use hashMap may cause ConcurrentModificationException, and ConcurrentHashMap can fix it.

Copy link
Contributor

Choose a reason for hiding this comment

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

Got it, maybe we can add more document about the details in the code.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Got it, maybe we can add more document about the details in the code.

ok, I will add more document later.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1. @novisfff could you use HUDI-4574 which is the original ticket for the timeline-server-based marker issue? I was investigating the issue and identified the same thread-safe problem.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

+1. @novisfff could you use HUDI-4574 which is the original ticket for the timeline-server-based marker issue? I was investigating the issue and identified the same thread-safe problem.

done.

Option<Integer> fileIndex = markerDirState.getNextFileIndexToUse();
if (!fileIndex.isPresent()) {
LOG.debug("All marker files are busy, skip batch processing of create marker requests in " + markerDir);
Expand Down