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

Fix RemoteFileUtil to download in parallel as expected. #5515

Merged
merged 6 commits into from
Nov 5, 2024
Merged
Changes from 3 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
35 changes: 33 additions & 2 deletions scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,11 @@
import java.nio.file.Paths;
import java.nio.file.StandardOpenOption;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.stream.Collectors;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.MatchResult.Metadata;
import org.apache.beam.sdk.io.fs.ResourceId;
Expand Down Expand Up @@ -103,10 +107,37 @@ public Path download(URI src) {
* @return {@link Path}s to the downloaded local files.
*/
public List<Path> download(List<URI> srcs) {
return download(srcs, CONCURRENCY_LEVEL);
}

/**
* Download a batch of remote {@link URI}s in parallel, using at most numThreads to do so.
* `numThreads` may not be larger than the number of available processors * 4.
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
* `numThreads` may not be larger than the number of available processors * 4.
* Specifying a `numThreads` greater than number of available processors * 4 will have the same effect as specifying `numThreads` equal to the available processors * 4.

Or maybe

Suggested change
* `numThreads` may not be larger than the number of available processors * 4.
* `numThreads` should at maximum be equal to the number of available processors * 4.

and then actually clamp the size of the thread pool

*
* @return {@link Path}s to the downloaded local files.
*/
public List<Path> download(List<URI> srcs, int numThreads) {
final ExecutorService executor = Executors.newFixedThreadPool(numThreads);
Copy link
Contributor

Choose a reason for hiding this comment

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

We should probably move this as transient member, initializing it in the constructor.
The class can implement AutoClosable to shutdown the thread pool.

This is mainly relevant for the FileDownloadDoFn where we don't want to create a new thread pool for every url batch.

try {
return paths.getAll(srcs).values().asList();
} catch (ExecutionException e) {
return executor
.invokeAll(
srcs.stream()
.map(url -> (Callable<Path>) () -> paths.get(url))
.collect(Collectors.toList()))
.stream()
.map(
f -> {
try {
return f.get();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
})
.collect(Collectors.toList());
} catch (InterruptedException e) {
throw new RuntimeException(e);
} finally {
executor.shutdown();
}
}

Expand Down