steveloughran commented on a change in pull request #34864:
URL: https://github.com/apache/spark/pull/34864#discussion_r778705352



##########
File path: 
external-shuffle-storage/src/main/java/org/apache/spark/starshuffle/StarBlockStoreClient.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.starshuffle;
+
+import org.apache.spark.SparkEnv;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.buffer.NioManagedBuffer;
+import org.apache.spark.network.netty.SparkTransportConf;
+import org.apache.spark.network.shuffle.*;
+import org.apache.spark.network.util.TransportConf;
+import org.apache.spark.storage.BlockId;
+import org.apache.spark.storage.ShuffleBlockId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class fetches shuffle blocks from external storage like S3
+ */
+public class StarBlockStoreClient extends BlockStoreClient {
+
+    private static final Logger logger = 
LoggerFactory.getLogger(StarBlockStoreClient.class);
+
+    // Fetch shuffle blocks from external shuffle storage.
+    // The shuffle location is encoded in the host argument. In the future, we 
should enhance
+    // Spark internal code to support abstraction of shuffle storage location.
+    @Override
+    public void fetchBlocks(String host, int port, String execId, String[] 
blockIds, BlockFetchingListener listener, DownloadFileManager 
downloadFileManager) {
+        for (int i = 0; i < blockIds.length; i++) {
+            String blockId = blockIds[i];
+            CompletableFuture.runAsync(() -> fetchBlock(host, execId, blockId, 
listener, downloadFileManager));
+        }
+    }
+
+    private void fetchBlock(String host, String execId, String blockIdStr, 
BlockFetchingListener listener, DownloadFileManager downloadFileManager) {
+        BlockId blockId = BlockId.apply(blockIdStr);
+        if (blockId instanceof ShuffleBlockId) {
+            ShuffleBlockId shuffleBlockId = (ShuffleBlockId)blockId;
+            StarMapResultFileInfo mapResultFileInfo = 
StarMapResultFileInfo.deserializeFromString(host);
+            long offset = 0;
+            for (int i = 0; i < shuffleBlockId.reduceId(); i++) {
+                offset += mapResultFileInfo.getPartitionLengths()[i];
+            }
+            long size = 
mapResultFileInfo.getPartitionLengths()[shuffleBlockId.reduceId()];
+            StarShuffleFileManager streamProvider = 
StarUtils.createShuffleFileManager(SparkEnv.get().conf(),
+                    mapResultFileInfo.getLocation());
+            if (downloadFileManager != null) {
+                try (InputStream inputStream = 
streamProvider.read(mapResultFileInfo.getLocation(), offset, size)) {
+                    TransportConf transportConf = 
SparkTransportConf.fromSparkConf(
+                            SparkEnv.get().conf(), "starShuffle", 1, 
Option.empty());
+                    DownloadFile downloadFile = 
downloadFileManager.createTempFile(transportConf);
+                    downloadFileManager.registerTempFileToClean(downloadFile);
+                    DownloadFileWritableChannel downloadFileWritableChannel = 
downloadFile.openForWriting();
+
+                    int bufferSize = 64 * 1024;
+                    byte[] bytes = new byte[bufferSize];
+                    int readBytes = 0;
+                    while (readBytes < size) {
+                        int toReadBytes = Math.min((int)size - readBytes, 
bufferSize);
+                        int n = inputStream.read(bytes, 0, toReadBytes);
+                        if (n == -1) {
+                            throw new RuntimeException(String.format(
+                                    "Failed to read file %s for shuffle block 
%s, hit end with remaining %s bytes",
+                                    mapResultFileInfo.getLocation(),
+                                    blockId,
+                                    size - readBytes));
+                        }
+                        readBytes += n;
+                        
downloadFileWritableChannel.write(ByteBuffer.wrap(bytes, 0, n));
+                    }
+                    ManagedBuffer managedBuffer = 
downloadFileWritableChannel.closeAndRead();
+                    listener.onBlockFetchSuccess(blockIdStr, managedBuffer);
+                } catch (IOException e) {
+                    throw new RuntimeException(String.format(

Review comment:
       include the inner exception text in the message and supply the exception 
as the inner exception in the constructor

##########
File path: external-shuffle-storage/pom.xml
##########
@@ -0,0 +1,143 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.spark</groupId>
+        <artifactId>spark-parent_2.12</artifactId>
+        <version>3.3.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>external-shuffle-storage_2.12</artifactId>
+    <packaging>jar</packaging>
+    <name>External Shuffle Storage</name>
+    <url>http://spark.apache.org/</url>
+
+    <properties>
+        <sbt.project.name>external-shuffle-storage</sbt.project.name>
+        <build.testJarPhase>none</build.testJarPhase>
+        <build.copyDependenciesPhase>package</build.copyDependenciesPhase>
+        <hadoop.deps.scope>provided</hadoop.deps.scope>
+        <hive.deps.scope>provided</hive.deps.scope>
+        <parquet.deps.scope>provided</parquet.deps.scope>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-math3</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scalacheck</groupId>
+            <artifactId>scalacheck_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>

Review comment:
       You should pull in spark-hadoop-cloud and so indirectly get its shaded 
full aws sdk. yes, it's big, but iat guarantees that it has a consistent set of 
its own dependencies (http client, jackson etc) and because it includes support 
for services like STS and s3 events, lets you add new features with guaranteed 
consistency of aws artifacts.

##########
File path: 
external-shuffle-storage/src/main/java/org/apache/spark/starshuffle/StarS3ShuffleFileManager.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.starshuffle;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.client.builder.ExecutorFactory;
+import com.amazonaws.event.ProgressEvent;
+import com.amazonaws.event.ProgressListener;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3ClientBuilder;
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.transfer.TransferManager;
+import com.amazonaws.services.s3.transfer.TransferManagerBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.SparkConf;
+import org.apache.spark.deploy.SparkHadoopUtil;
+import org.apache.spark.network.util.LimitedInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.net.URI;
+import java.util.UUID;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class read/write shuffle file on external storage like S3.
+ */
+public class StarS3ShuffleFileManager implements StarShuffleFileManager {
+    private static final Logger logger = 
LoggerFactory.getLogger(StarS3ShuffleFileManager.class);
+
+    // TODO make following values configurable
+    public final static int S3_PUT_TIMEOUT_MILLISEC = 180 * 1000;
+
+    // Following constants are copied from:
+    // 
https://github.com/apache/hadoop/blob/6c6d1b64d4a7cd5288fcded78043acaf23228f96/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+    public static final long DEFAULT_MULTIPART_SIZE = 67108864; // 64M
+    public static final long DEFAULT_MIN_MULTIPART_THRESHOLD = 134217728; // 
128M
+    public static final String MAX_THREADS = "fs.s3a.threads.max";
+    public static final int DEFAULT_MAX_THREADS = 10;
+    public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime";
+    public static final int DEFAULT_KEEPALIVE_TIME = 60;
+
+    public static final String AWS_REGION = "fs.s3a.endpoint.region";
+    public static final String DEFAULT_AWS_REGION = 
Regions.US_WEST_2.getName();
+
+    private static TransferManager transferManager;
+    private static Object transferManagerLock = new Object();

Review comment:
       final

##########
File path: 
external-shuffle-storage/src/main/java/org/apache/spark/starshuffle/StarBypassMergeSortShuffleWriter.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.starshuffle;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.spark.Partitioner;
+import org.apache.spark.ShuffleDependency;
+import org.apache.spark.SparkConf;
+import org.apache.spark.internal.config.package$;
+import org.apache.spark.scheduler.MapStatus;
+import org.apache.spark.serializer.Serializer;
+import org.apache.spark.serializer.SerializerInstance;
+import org.apache.spark.shuffle.ShuffleWriteMetricsReporter;
+import org.apache.spark.shuffle.ShuffleWriter;
+import org.apache.spark.shuffle.StarBypassMergeSortShuffleHandle;
+import org.apache.spark.shuffle.StarOpts;
+import org.apache.spark.shuffle.api.ShuffleExecutorComponents;
+import org.apache.spark.shuffle.api.ShuffleMapOutputWriter;
+import org.apache.spark.shuffle.sort.SortShuffleManager;
+import org.apache.spark.storage.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.None$;
+import scala.Option;
+import scala.Product2;
+import scala.Tuple2;
+import scala.collection.Iterator;
+
+import javax.annotation.Nullable;
+import java.io.*;

Review comment:
       bit brittle against jvm releases adding new classes here

##########
File path: 
external-shuffle-storage/src/main/java/org/apache/spark/starshuffle/StarS3ShuffleFileManager.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.starshuffle;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.client.builder.ExecutorFactory;
+import com.amazonaws.event.ProgressEvent;
+import com.amazonaws.event.ProgressListener;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3ClientBuilder;
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.transfer.TransferManager;
+import com.amazonaws.services.s3.transfer.TransferManagerBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.SparkConf;
+import org.apache.spark.deploy.SparkHadoopUtil;
+import org.apache.spark.network.util.LimitedInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.net.URI;
+import java.util.UUID;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class read/write shuffle file on external storage like S3.
+ */
+public class StarS3ShuffleFileManager implements StarShuffleFileManager {
+    private static final Logger logger = 
LoggerFactory.getLogger(StarS3ShuffleFileManager.class);
+
+    // TODO make following values configurable
+    public final static int S3_PUT_TIMEOUT_MILLISEC = 180 * 1000;
+
+    // Following constants are copied from:
+    // 
https://github.com/apache/hadoop/blob/6c6d1b64d4a7cd5288fcded78043acaf23228f96/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+    public static final long DEFAULT_MULTIPART_SIZE = 67108864; // 64M
+    public static final long DEFAULT_MIN_MULTIPART_THRESHOLD = 134217728; // 
128M
+    public static final String MAX_THREADS = "fs.s3a.threads.max";
+    public static final int DEFAULT_MAX_THREADS = 10;
+    public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime";
+    public static final int DEFAULT_KEEPALIVE_TIME = 60;
+
+    public static final String AWS_REGION = "fs.s3a.endpoint.region";
+    public static final String DEFAULT_AWS_REGION = 
Regions.US_WEST_2.getName();
+
+    private static TransferManager transferManager;
+    private static Object transferManagerLock = new Object();
+
+    private final String awsRegion;
+    private final int maxThreads;
+    private final long keepAliveTime;
+
+    public StarS3ShuffleFileManager(SparkConf conf) {
+        Configuration hadoopConf = 
SparkHadoopUtil.get().newConfiguration(conf);
+
+        awsRegion = hadoopConf.get(AWS_REGION, DEFAULT_AWS_REGION);
+
+        int threads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
+        if (threads < 2) {
+            logger.warn(MAX_THREADS + " must be at least 2: forcing to 2.");
+            threads = 2;
+        }
+        maxThreads = threads;
+
+        keepAliveTime = conf.getLong(KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME);
+    }
+
+    @Override
+    public String createFile(String root) {
+        if (!root.endsWith("/")) {
+            root = root + "/";
+        }
+        String fileName = String.format("shuffle-%s.data", UUID.randomUUID());
+        return root + fileName;
+    }
+
+    @Override
+    public void write(InputStream data, long size, String file) {
+        logger.info("Writing to shuffle file: {}", file);
+        writeS3(data, size, file);
+    }
+
+    @Override
+    public InputStream read(String file, long offset, long size) {
+        logger.info("Opening shuffle file: {}, offset: {}, size: {}", file, 
offset, size);
+        return readS3(file, offset, size);
+    }
+
+    private void writeS3(InputStream inputStream, long size, String s3Url) {
+        logger.info("Uploading shuffle file to s3: {}, size: {}", s3Url, size);
+
+        S3BucketAndKey bucketAndKey = S3BucketAndKey.getFromUrl(s3Url);
+        String bucket = bucketAndKey.getBucket();
+        String key = bucketAndKey.getKey();
+
+        TransferManager transferManager = getTransferManager();
+
+        ObjectMetadata metadata = new ObjectMetadata();
+        metadata.setContentType("application/octet-stream");
+        metadata.setContentLength(size);
+
+        PutObjectRequest request = new PutObjectRequest(bucket,
+                key,
+                inputStream,
+                metadata);
+
+        AtomicLong totalTransferredBytes = new AtomicLong(0);
+
+        request.setGeneralProgressListener(new ProgressListener() {
+            private long lastLogTime = 0;
+
+            @Override
+            public void progressChanged(ProgressEvent progressEvent) {
+                long count = progressEvent.getBytesTransferred();
+                long total = totalTransferredBytes.addAndGet(count);
+                long currentTime = System.currentTimeMillis();
+                long logInterval = 10000;
+                if (currentTime - lastLogTime >= logInterval) {
+                    logger.info("S3 upload progress: {}, recent transferred {} 
bytes, total transferred {}", key, count, total);
+                    lastLogTime = currentTime;
+                }
+            }
+        });
+
+        // 
https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/best-practices.html
+        request.getRequestClientOptions().setReadLimit((int) 
DEFAULT_MULTIPART_SIZE + 1);
+        request.setSdkRequestTimeout(S3_PUT_TIMEOUT_MILLISEC);
+        request.setSdkClientExecutionTimeout(S3_PUT_TIMEOUT_MILLISEC);
+        try {
+            long startTime = System.currentTimeMillis();
+            transferManager.upload(request).waitForCompletion();
+            long duration = System.currentTimeMillis() - startTime;
+            double mbs = 0;
+            if (duration != 0) {
+                mbs = ((double) size) / (1000 * 1000) / ((double) duration / 
1000);
+            }
+            logger.info("S3 upload finished: {}, file size: {} bytes, total 
transferred: {}, throughput: {} mbs",
+                    s3Url, size, totalTransferredBytes.get(), mbs);
+        } catch (InterruptedException e) {
+            throw new RuntimeException("Failed to upload to s3: " + key, e);
+        } finally {
+            transferManager.shutdownNow();
+        }
+    }
+
+    private InputStream readS3(String s3Url, long offset, long size) {
+        logger.info("Downloading shuffle file from s3: {}, size: {}", s3Url, 
size);
+
+        S3BucketAndKey bucketAndKey = S3BucketAndKey.getFromUrl(s3Url);
+
+        File downloadTempFile;
+        try {
+            downloadTempFile = File.createTempFile("shuffle-download", 
".data");
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to create temp file for 
downloading shuffle file");
+        }
+
+        TransferManager transferManager = getTransferManager();
+
+        GetObjectRequest getObjectRequest = new 
GetObjectRequest(bucketAndKey.getBucket(), bucketAndKey.getKey())
+                .withRange(offset, offset + size);
+
+        AtomicLong totalTransferredBytes = new AtomicLong(0);
+
+        getObjectRequest.setGeneralProgressListener(new ProgressListener() {
+            private long lastLogTime = 0;
+
+            @Override
+            public void progressChanged(ProgressEvent progressEvent) {
+                long count = progressEvent.getBytesTransferred();
+                long total = totalTransferredBytes.addAndGet(count);
+                long currentTime = System.currentTimeMillis();
+                long logInterval = 10000;
+                if (currentTime - lastLogTime >= logInterval) {
+                    logger.info("S3 download progress: {}, recent transferred 
{} bytes, total transferred {}", s3Url, count, total);
+                    lastLogTime = currentTime;
+                }
+            }
+        });
+
+        try {
+            long startTime = System.currentTimeMillis();
+            transferManager.download(getObjectRequest, 
downloadTempFile).waitForCompletion();
+            long duration = System.currentTimeMillis() - startTime;
+            double mbs = 0;
+            if (duration != 0) {
+                mbs = ((double) size) / (1000 * 1000) / ((double) duration / 
1000);
+            }
+            logger.info("S3 download finished: {}, file size: {} bytes, total 
transferred: {}, throughput: {} mbs",
+                    s3Url, size, totalTransferredBytes.get(), mbs);
+        } catch (InterruptedException e) {
+            throw new RuntimeException(String.format(
+                    "Failed to download shuffle file %s", s3Url));
+        } finally {
+            transferManager.shutdownNow();

Review comment:
       What if the transfer manager is reused?

##########
File path: 
external-shuffle-storage/src/main/java/org/apache/spark/starshuffle/StarBlockStoreClient.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.starshuffle;
+
+import org.apache.spark.SparkEnv;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.buffer.NioManagedBuffer;
+import org.apache.spark.network.netty.SparkTransportConf;
+import org.apache.spark.network.shuffle.*;
+import org.apache.spark.network.util.TransportConf;
+import org.apache.spark.storage.BlockId;
+import org.apache.spark.storage.ShuffleBlockId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class fetches shuffle blocks from external storage like S3
+ */
+public class StarBlockStoreClient extends BlockStoreClient {
+
+    private static final Logger logger = 
LoggerFactory.getLogger(StarBlockStoreClient.class);
+
+    // Fetch shuffle blocks from external shuffle storage.
+    // The shuffle location is encoded in the host argument. In the future, we 
should enhance
+    // Spark internal code to support abstraction of shuffle storage location.
+    @Override
+    public void fetchBlocks(String host, int port, String execId, String[] 
blockIds, BlockFetchingListener listener, DownloadFileManager 
downloadFileManager) {
+        for (int i = 0; i < blockIds.length; i++) {
+            String blockId = blockIds[i];
+            CompletableFuture.runAsync(() -> fetchBlock(host, execId, blockId, 
listener, downloadFileManager));
+        }
+    }
+
+    private void fetchBlock(String host, String execId, String blockIdStr, 
BlockFetchingListener listener, DownloadFileManager downloadFileManager) {
+        BlockId blockId = BlockId.apply(blockIdStr);
+        if (blockId instanceof ShuffleBlockId) {
+            ShuffleBlockId shuffleBlockId = (ShuffleBlockId)blockId;
+            StarMapResultFileInfo mapResultFileInfo = 
StarMapResultFileInfo.deserializeFromString(host);
+            long offset = 0;
+            for (int i = 0; i < shuffleBlockId.reduceId(); i++) {
+                offset += mapResultFileInfo.getPartitionLengths()[i];
+            }
+            long size = 
mapResultFileInfo.getPartitionLengths()[shuffleBlockId.reduceId()];
+            StarShuffleFileManager streamProvider = 
StarUtils.createShuffleFileManager(SparkEnv.get().conf(),
+                    mapResultFileInfo.getLocation());
+            if (downloadFileManager != null) {
+                try (InputStream inputStream = 
streamProvider.read(mapResultFileInfo.getLocation(), offset, size)) {
+                    TransportConf transportConf = 
SparkTransportConf.fromSparkConf(
+                            SparkEnv.get().conf(), "starShuffle", 1, 
Option.empty());
+                    DownloadFile downloadFile = 
downloadFileManager.createTempFile(transportConf);
+                    downloadFileManager.registerTempFileToClean(downloadFile);
+                    DownloadFileWritableChannel downloadFileWritableChannel = 
downloadFile.openForWriting();
+
+                    int bufferSize = 64 * 1024;
+                    byte[] bytes = new byte[bufferSize];
+                    int readBytes = 0;
+                    while (readBytes < size) {
+                        int toReadBytes = Math.min((int)size - readBytes, 
bufferSize);
+                        int n = inputStream.read(bytes, 0, toReadBytes);
+                        if (n == -1) {
+                            throw new RuntimeException(String.format(
+                                    "Failed to read file %s for shuffle block 
%s, hit end with remaining %s bytes",
+                                    mapResultFileInfo.getLocation(),
+                                    blockId,
+                                    size - readBytes));
+                        }
+                        readBytes += n;
+                        
downloadFileWritableChannel.write(ByteBuffer.wrap(bytes, 0, n));
+                    }
+                    ManagedBuffer managedBuffer = 
downloadFileWritableChannel.closeAndRead();
+                    listener.onBlockFetchSuccess(blockIdStr, managedBuffer);
+                } catch (IOException e) {
+                    throw new RuntimeException(String.format(
+                            "Failed to read file %s for shuffle block %s",
+                            mapResultFileInfo.getLocation(),
+                            blockId));
+                }
+            } else {
+                try (InputStream inputStream = 
streamProvider.read(mapResultFileInfo.getLocation(), offset, size)) {
+                    ByteBuffer byteBuffer = ByteBuffer.allocate((int)size);
+                    int b = inputStream.read();
+                    while (b != -1) {
+                        byteBuffer.put((byte)b);
+                        if (byteBuffer.position() == size) {
+                            break;
+                        }
+                        b = inputStream.read();
+                    }
+                    byteBuffer.flip();
+                    NioManagedBuffer managedBuffer = new 
NioManagedBuffer(byteBuffer);
+                    listener.onBlockFetchSuccess(blockIdStr, managedBuffer);
+                } catch (IOException e) {
+                    throw new RuntimeException(String.format(

Review comment:
       Again, pass on inner exception details.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to