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-8782] BulkInsertWriterHelper parallel close #12518

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
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 @@ -25,6 +25,7 @@
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.configuration.OptionsResolver;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.row.HoodieRowDataCreateHandle;
import org.apache.hudi.metrics.FlinkStreamWriteMetrics;
import org.apache.hudi.table.HoodieTable;
Expand All @@ -45,6 +46,13 @@
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

import static org.apache.hudi.common.util.FutureUtils.allOf;

/**
* Helper class for bulk insert used by Flink.
Expand Down Expand Up @@ -157,9 +165,25 @@ private HoodieRowDataCreateHandle getRowCreateHandle(String partitionPath) throw
}

public void close() throws IOException {
for (HoodieRowDataCreateHandle rowCreateHandle : handles.values()) {
LOG.info("Closing bulk insert file " + rowCreateHandle.getFileName());
writeStatusList.add(closeWriteHandle(rowCreateHandle));
ExecutorService executorService = Executors.newFixedThreadPool(handles.size());
allOf(handles.values().stream()
.map(rowCreateHandle -> CompletableFuture.supplyAsync(() -> {
try {
LOG.info("Closing bulk insert file " + rowCreateHandle.getFileName());
return rowCreateHandle.close();
} catch (IOException e) {
throw new HoodieIOException("IOE during rowCreateHandle.close()", e);
}
}, executorService))
.collect(Collectors.toList())
).whenComplete((result, throwable) -> {
writeStatusList.addAll(result);
}).join();
try {
executorService.shutdown();
executorService.awaitTermination(24, TimeUnit.DAYS);
Copy link
Contributor

Choose a reason for hiding this comment

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

24 days?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sorry! 24 hours. this indicates a long timeout , aiming to waiting for all handles closed.

Copy link
Contributor

Choose a reason for hiding this comment

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

Still too long, the close should be fast right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Still too long, the close should be fast right?

I agree, close should be fast. here awaitTermination is actually a timeout, 24 Hours is too long , how about wait for a Flink CP interval?

} catch (InterruptedException e) {
throw new RuntimeException(e);
}
handles.clear();
handle = null;
Expand Down
Loading