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 4 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
34 changes: 27 additions & 7 deletions scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,10 @@

package com.spotify.scio.util;

import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
Expand All @@ -29,11 +33,13 @@
import java.nio.file.Paths;
import java.nio.file.StandardOpenOption;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.*;
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;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.util.MimeTypes;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
Expand All @@ -44,17 +50,16 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* A utility class for handling remote file systems designed to be used in a {@link
* org.apache.beam.sdk.transforms.DoFn}.
*/
/** A utility class for handling remote file systems designed to be used in a {@link DoFn}. */
public class RemoteFileUtil implements Serializable {
RustedBones marked this conversation as resolved.
Show resolved Hide resolved

private static final Logger LOG = LoggerFactory.getLogger(RemoteFileUtil.class);

private static final int CONCURRENCY_LEVEL = Runtime.getRuntime().availableProcessors() * 4;
private static final int HASH_LENGTH = 8;

private transient ListeningExecutorService executorService;

// Mapping of remote sources to local destinations
private static final LoadingCache<URI, Path> paths =
CacheBuilder.newBuilder()
Expand All @@ -74,6 +79,16 @@ public static RemoteFileUtil create(PipelineOptions options) {
return new RemoteFileUtil();
}

private synchronized ListeningExecutorService getExecutorService() {
if (executorService == null) {
ThreadPoolExecutor threadPool =
(ThreadPoolExecutor) Executors.newFixedThreadPool(CONCURRENCY_LEVEL);
executorService =
MoreExecutors.listeningDecorator(MoreExecutors.getExitingExecutorService(threadPool));
}
return executorService;
}

/** Check if a remote {@link URI} exists. */
public boolean remoteExists(URI uri) throws IOException {
try {
Expand Down Expand Up @@ -103,9 +118,14 @@ public Path download(URI src) {
* @return {@link Path}s to the downloaded local files.
*/
public List<Path> download(List<URI> srcs) {
ListeningExecutorService executor = getExecutorService();
List<ListenableFuture<Path>> futures =
srcs.stream()
.map(uri -> executor.submit(() -> paths.get(uri)))
.collect(Collectors.toList());
try {
return paths.getAll(srcs).values().asList();
} catch (ExecutionException e) {
return Futures.allAsList(futures).get();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
}
Expand Down