You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by jinxing64 <gi...@git.apache.org> on 2017/07/07 15:38:04 UTC

[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

GitHub user jinxing64 opened a pull request:

    https://github.com/apache/spark/pull/18565

    [SPARK-21342] Fix DownloadCallback to work well with RetryingBlockFetcher.

    ## What changes were proposed in this pull request?
    
    When `RetryingBlockFetcher` retries fetching blocks. There could be two `DownloadCallback`s download the same content to the same target file. It could cause `ShuffleBlockFetcherIterator` reading a partial result. This pr proposes to write a tmp file and rename it to the target file when finish.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/jinxing64/spark SPARK-21342

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/18565.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #18565
    
----
commit da65427a94b24b2ad0a6ecf79ac1980dd458c684
Author: jinxing <ji...@126.com>
Date:   2017-07-07T15:29:58Z

    Fix DownloadCallback to work well with RetryingBlockFetcher.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79390 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79390/testReport)** for PR 18565 at commit [`6307a62`](https://github.com/apache/spark/commit/6307a6279ed7bf3680c7ae3836128bdfdf228697).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126259778
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -151,15 +152,27 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
         }
       }
     
    +  private static synchronized boolean renameFile(File src, File dest) {
    +    if (dest.exists()) {
    +      if (!dest.delete()) {
    --- End diff --
    
    Here is what I suggest:
    ```
        private boolean rename() {
          synchronized (targetFile) {
            // Lock to avoid renaming at the same time. `File.renameTo` doesn't promise to fail on
            // all file systems when the target file exists.
            if (!targetFile.exists()) {
              if (!tmpFile.renameTo(targetFile)) {
                if (!targetFile.exists()) {
                  return false;
                }
              }
            }
          }
          return true;
        }
    
        @Override
        public void onComplete(String streamId) throws IOException {
          channel.close();
          if (!rename()) {
            onFailure(streamId, new Exception("Failed renaming " + tmpFile.getAbsolutePath() + " to " + targetFile.getAbsolutePath()));
            return;
          }
          ManagedBuffer buffer = new FileSegmentManagedBuffer(transportConf, targetFile, 0,
            targetFile.length());
          tmpFile.delete();
          listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer);
        }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126306207
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -58,29 +59,41 @@
       private final BlockFetchingListener listener;
       private final ChunkReceivedCallback chunkCallback;
       private TransportConf transportConf = null;
    -  private File[] shuffleFiles = null;
    +  private Boolean toDisk;
    +  private Supplier<File> tmpFileCreater;
    +  private Supplier<Boolean> shuffleBlockFetcherIteratorIsZombie;
     
       private StreamHandle streamHandle = null;
     
       public OneForOneBlockFetcher(
    +    TransportClient client,
    +    String appId,
    +    String execId,
    +    String[] blockIds,
    +    BlockFetchingListener listener,
    +    TransportConf transportConf) {
    +    this(client, appId, execId, blockIds, listener, transportConf, false, null, null);
    +  }
    +
    +  public OneForOneBlockFetcher(
           TransportClient client,
           String appId,
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
           TransportConf transportConf,
    -      File[] shuffleFiles) {
    +      Boolean toDisk,
    +      Supplier<File> tmpFileCreater,
    +      Supplier<Boolean> shuffleBlockFetcherIteratorIsZombie) {
    --- End diff --
    
    I will refine.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79420/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/18565


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126285972
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java ---
    @@ -42,5 +42,13 @@ public abstract void fetchBlocks(
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
    -      File[] shuffleFiles);
    +      boolean toDisk,
    +      TmpFileCreater tmpFileCreater);
    +
    +  /**
    +   * An interface to provide approach to create tmp file.
    +   */
    +  public interface TmpFileCreater {
    +    File createTempBlock();
    --- End diff --
    
    `createTempFile` may be a better name?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    In current change:
    1. I'm using `java.util.function.Supplier` instead of `TmpFileCreater`
    2. Pass `shuffleBlockFetcherIteratorIsZombie` from `ShuffleBlockFetcherIterator` to `OneForOneBlockFetcher`;
    3. Succeeded files will be cleanup in `ShuffleBlockFetcherIterator`, while others will be deleted in `OneForOneBlockFetcher`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79409 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79409/testReport)** for PR 18565 at commit [`9aeb090`](https://github.com/apache/spark/commit/9aeb09072b735cba3d466ecb965443b713872744).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126291505
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala ---
    @@ -53,6 +53,7 @@ private[spark] class NettyBlockTransferService(
       private val serializer = new JavaSerializer(conf)
       private val authEnabled = securityManager.isAuthenticationEnabled()
       private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numCores)
    +  private val blockFetchers = collection.mutable.HashSet[OneForOneBlockFetcher]()
    --- End diff --
    
    this is not thread safe. And it's a memory leak because it won't be cleaned until the executor is shutting down.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79422/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79389 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79389/testReport)** for PR 18565 at commit [`3b68b2c`](https://github.com/apache/spark/commit/3b68b2c356a611b3490f11112222424788139811).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126286235
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -151,14 +167,21 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
         }
       }
     
    +  public void cleanup() {
    +    for (File file: shuffleFiles) {
    +      file.delete();
    +    }
    +  }
    +
       private class DownloadCallback implements StreamCallback {
     
         private WritableByteChannel channel = null;
         private File targetFile = null;
         private int chunkIndex;
     
    -    DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    -      this.targetFile = targetFile;
    +    DownloadCallback(int chunkIndex) throws IOException {
    +      this.targetFile = tmpFileCreater.createTempBlock();
    +      shuffleFiles.add(targetFile);
    --- End diff --
    
    ditto, we will create unique temp files, so `shuffleFiles` can be a list instead of set.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    @jinxing64 I submitted https://github.com/jinxing64/spark/pull/1 to your repo to fix a potential file leak. Otherwise, this looks good to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126301763
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -58,29 +59,41 @@
       private final BlockFetchingListener listener;
       private final ChunkReceivedCallback chunkCallback;
       private TransportConf transportConf = null;
    -  private File[] shuffleFiles = null;
    +  private Boolean toDisk;
    +  private Supplier<File> tmpFileCreater;
    +  private Supplier<Boolean> shuffleBlockFetcherIteratorIsZombie;
     
       private StreamHandle streamHandle = null;
     
       public OneForOneBlockFetcher(
    +    TransportClient client,
    +    String appId,
    +    String execId,
    +    String[] blockIds,
    +    BlockFetchingListener listener,
    +    TransportConf transportConf) {
    +    this(client, appId, execId, blockIds, listener, transportConf, false, null, null);
    +  }
    +
    +  public OneForOneBlockFetcher(
           TransportClient client,
           String appId,
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
           TransportConf transportConf,
    -      File[] shuffleFiles) {
    +      Boolean toDisk,
    +      Supplier<File> tmpFileCreater,
    +      Supplier<Boolean> shuffleBlockFetcherIteratorIsZombie) {
    --- End diff --
    
    Since the network module doesn't depend on the core module, we should hide the `ShuffleBlockFetcherIterator` from network module. How about we name it `canCallerSideDeleteFile`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126301738
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -58,29 +59,41 @@
       private final BlockFetchingListener listener;
       private final ChunkReceivedCallback chunkCallback;
       private TransportConf transportConf = null;
    -  private File[] shuffleFiles = null;
    +  private Boolean toDisk;
    +  private Supplier<File> tmpFileCreater;
    +  private Supplier<Boolean> shuffleBlockFetcherIteratorIsZombie;
     
       private StreamHandle streamHandle = null;
     
       public OneForOneBlockFetcher(
    +    TransportClient client,
    +    String appId,
    +    String execId,
    +    String[] blockIds,
    +    BlockFetchingListener listener,
    +    TransportConf transportConf) {
    +    this(client, appId, execId, blockIds, listener, transportConf, false, null, null);
    +  }
    +
    +  public OneForOneBlockFetcher(
           TransportClient client,
           String appId,
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
           TransportConf transportConf,
    -      File[] shuffleFiles) {
    +      Boolean toDisk,
    +      Supplier<File> tmpFileCreater,
    +      Supplier<Boolean> shuffleBlockFetcherIteratorIsZombie) {
         this.client = client;
         this.openMessage = new OpenBlocks(appId, execId, blockIds);
         this.blockIds = blockIds;
         this.listener = listener;
         this.chunkCallback = new ChunkCallback();
         this.transportConf = transportConf;
    -    if (shuffleFiles != null) {
    -      this.shuffleFiles = shuffleFiles;
    -      assert this.shuffleFiles.length == blockIds.length:
    -        "Number of shuffle files should equal to blocks";
    -    }
    +    this.toDisk = toDisk;
    +    this.tmpFileCreater = tmpFileCreater;
    +    this.shuffleBlockFetcherIteratorIsZombie = shuffleBlockFetcherIteratorIsZombie;
    --- End diff --
    
    shall we add an assert that, if `toDisk` is true, `tmpFileCreater` and `shuffleBlockFetcherIteratorIsZombie` can not be null. Or alternatively, we can drop the `toDisk` parameter, and use `toDisk = tmpFileCreater != null`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126242905
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -151,15 +152,27 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
         }
       }
     
    +  private static synchronized boolean renameFile(File src, File dest) {
    +    if (dest.exists()) {
    +      if (!dest.delete()) {
    --- End diff --
    
    In addition, you don't need to use `synchronized` on a static method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126277863
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -151,15 +152,27 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
         }
       }
     
    +  private static synchronized boolean renameFile(File src, File dest) {
    +    if (dest.exists()) {
    +      if (!dest.delete()) {
    --- End diff --
    
    I was using `synchronized static` because there could be multiple `OneForOneBlockFetcher` doing the rename work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79378 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79378/testReport)** for PR 18565 at commit [`dc7acfe`](https://github.com/apache/spark/commit/dc7acfed9b3e2336ee75333247f434e5db6645d1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79335 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79335/testReport)** for PR 18565 at commit [`da65427`](https://github.com/apache/spark/commit/da65427a94b24b2ad0a6ecf79ac1980dd458c684).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126306193
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -119,9 +132,9 @@ public void onSuccess(ByteBuffer response) {
               // Immediately request all chunks -- we expect that the total size of the request is
               // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]].
               for (int i = 0; i < streamHandle.numChunks; i++) {
    -            if (shuffleFiles != null) {
    +            if (toDisk) {
    --- End diff --
    
    Good idea.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79390/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    cc @zsxwing @cloud-fan @jiangxb1987 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Jenkins, retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79438 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79438/testReport)** for PR 18565 at commit [`15001a6`](https://github.com/apache/spark/commit/15001a6dac4b02ef408a7e800a146e776a204435).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79378/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79408/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79435 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79435/testReport)** for PR 18565 at commit [`bb82dd3`](https://github.com/apache/spark/commit/bb82dd390ad63e5ebf93e543d903e76e7308b5ef).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79409 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79409/testReport)** for PR 18565 at commit [`9aeb090`](https://github.com/apache/spark/commit/9aeb09072b735cba3d466ecb965443b713872744).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126242576
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -151,15 +152,27 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
         }
       }
     
    +  private static synchronized boolean renameFile(File src, File dest) {
    +    if (dest.exists()) {
    +      if (!dest.delete()) {
    --- End diff --
    
    Why delete the existing file? If it's already there, then it must be done by another `DownloadCallback`. It should be fine to just mark DownloadCallback successful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126285966
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java ---
    @@ -42,5 +42,13 @@ public abstract void fetchBlocks(
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
    -      File[] shuffleFiles);
    +      boolean toDisk,
    +      TmpFileCreater tmpFileCreater);
    +
    +  /**
    +   * An interface to provide approach to create tmp file.
    +   */
    +  public interface TmpFileCreater {
    --- End diff --
    
    shall we move it to a separated file? We may use it outside of `ShuffleClient` in the future.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79438 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79438/testReport)** for PR 18565 at commit [`15001a6`](https://github.com/apache/spark/commit/15001a6dac4b02ef408a7e800a146e776a204435).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79335 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79335/testReport)** for PR 18565 at commit [`da65427`](https://github.com/apache/spark/commit/da65427a94b24b2ad0a6ecf79ac1980dd458c684).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126285842
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java ---
    @@ -91,15 +95,18 @@ public void fetchBlocks(
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
    -      File[] shuffleFiles) {
    +      boolean toDisk,
    +      TmpFileCreater tmpFileCreater) {
    --- End diff --
    
    we can use null `tmpFileCreater` to represent not fetching to disk


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79422/testReport)** for PR 18565 at commit [`ef9f994`](https://github.com/apache/spark/commit/ef9f994ab51c43583b0160e5451c256498fd6fec).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79390 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79390/testReport)** for PR 18565 at commit [`6307a62`](https://github.com/apache/spark/commit/6307a6279ed7bf3680c7ae3836128bdfdf228697).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79452 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79452/testReport)** for PR 18565 at commit [`15001a6`](https://github.com/apache/spark/commit/15001a6dac4b02ef408a7e800a146e776a204435).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126281278
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java ---
    @@ -91,15 +95,17 @@ public void fetchBlocks(
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
    -      File[] shuffleFiles) {
    +      boolean toDisk) {
    --- End diff --
    
    shall we pass in `TmpFileCreater` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126178968
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -151,15 +152,27 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
         }
       }
     
    +  private static synchronized boolean renameFile(File src, File dest) {
    +    if (dest.exists()) {
    +      if (!dest.delete()) {
    +        return false;
    +      }
    +    }
    +    return src.renameTo(dest);
    +  }
    +
       private class DownloadCallback implements StreamCallback {
     
         private WritableByteChannel channel = null;
         private File targetFile = null;
    +    private File tmpFile = null;
         private int chunkIndex;
     
    -    DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    -      this.targetFile = targetFile;
    -      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +    DownloadCallback(int chunkIndex) throws IOException {
    --- End diff --
    
    This is unrelated, just remove a redundant param.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79409/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79408 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79408/testReport)** for PR 18565 at commit [`31664d4`](https://github.com/apache/spark/commit/31664d479b16de57be57189219e236021bfc1544).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    LGTM except 2 minor comments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79452 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79452/testReport)** for PR 18565 at commit [`15001a6`](https://github.com/apache/spark/commit/15001a6dac4b02ef408a7e800a146e776a204435).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126285935
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java ---
    @@ -91,15 +95,18 @@ public void fetchBlocks(
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
    -      File[] shuffleFiles) {
    +      boolean toDisk,
    +      TmpFileCreater tmpFileCreater) {
         checkInit();
         logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId);
         try {
           RetryingBlockFetcher.BlockFetchStarter blockFetchStarter =
               (blockIds1, listener1) -> {
                 TransportClient client = clientFactory.createClient(host, port);
    -            new OneForOneBlockFetcher(client, appId, execId, blockIds1, listener1, conf,
    -              shuffleFiles).start();
    +            OneForOneBlockFetcher blockFetcher = new OneForOneBlockFetcher(client, appId, execId,
    +              blockIds1, listener1, conf, toDisk, tmpFileCreater);
    +            blockfetchers.add(blockFetcher);
    --- End diff --
    
    `OneForOneBlockFetcher` doesn't implement `equals` and `hashCode`, so 2 `OneForOneBlockFetcher`s are considered equal if and only if they are same instance.
    
    Here `blockfetchers.add(blockFetcher)` always add a new instance, so I don't thing using set to store block fetchers makes sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79438/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79435 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79435/testReport)** for PR 18565 at commit [`bb82dd3`](https://github.com/apache/spark/commit/bb82dd390ad63e5ebf93e543d903e76e7308b5ef).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79420 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79420/testReport)** for PR 18565 at commit [`5d2f98c`](https://github.com/apache/spark/commit/5d2f98c93d8caca67a53877dc6a3f539b3ee1cc3).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126305755
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -119,9 +132,9 @@ public void onSuccess(ByteBuffer response) {
               // Immediately request all chunks -- we expect that the total size of the request is
               // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]].
               for (int i = 0; i < streamHandle.numChunks; i++) {
    -            if (shuffleFiles != null) {
    +            if (toDisk) {
    --- End diff --
    
    We can use `tmpFileCreater != null` instead, so we may remove all the `toDisk` params.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by jinxing64 <gi...@git.apache.org>.
Github user jinxing64 commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    There are some corner cases because of creating and deleting in both `ShuffleBlockFetcherIterator` and `OneForOneBlockFetcher`. It seems no need to pass the `shuffleFiles` from `ShuffleBlockFetcherIterator` to `ShuffleClient` and `OneForOneBlockFetcher`. Now I changed create and delete files only in `OneForOneBlockFetcher`. We don't do renaming. Every `DownloadCallback` will have their own target file. All target files will be deleted when blockManager stop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    thanks, merging to master/2.2!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79410 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79410/testReport)** for PR 18565 at commit [`d489ba2`](https://github.com/apache/spark/commit/d489ba25bc1b92d2dbc6100ad789e5b512fb13b1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79389/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79410/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79452/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79389 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79389/testReport)** for PR 18565 at commit [`3b68b2c`](https://github.com/apache/spark/commit/3b68b2c356a611b3490f11112222424788139811).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79420 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79420/testReport)** for PR 18565 at commit [`5d2f98c`](https://github.com/apache/spark/commit/5d2f98c93d8caca67a53877dc6a3f539b3ee1cc3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79410 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79410/testReport)** for PR 18565 at commit [`d489ba2`](https://github.com/apache/spark/commit/d489ba25bc1b92d2dbc6100ad789e5b512fb13b1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79408 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79408/testReport)** for PR 18565 at commit [`31664d4`](https://github.com/apache/spark/commit/31664d479b16de57be57189219e236021bfc1544).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79422 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79422/testReport)** for PR 18565 at commit [`ef9f994`](https://github.com/apache/spark/commit/ef9f994ab51c43583b0160e5451c256498fd6fec).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79435/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79335/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18565: [SPARK-21342] Fix DownloadCallback to work well with Ret...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18565
  
    **[Test build #79378 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79378/testReport)** for PR 18565 at commit [`dc7acfe`](https://github.com/apache/spark/commit/dc7acfed9b3e2336ee75333247f434e5db6645d1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18565: [SPARK-21342] Fix DownloadCallback to work well w...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18565#discussion_r126286273
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java ---
    @@ -42,5 +42,13 @@ public abstract void fetchBlocks(
           String execId,
           String[] blockIds,
           BlockFetchingListener listener,
    -      File[] shuffleFiles);
    +      boolean toDisk,
    +      TmpFileCreater tmpFileCreater);
    +
    +  /**
    +   * An interface to provide approach to create tmp file.
    +   */
    +  public interface TmpFileCreater {
    --- End diff --
    
    Actually we may not need this, just let `ExternalShuffleClient.fetchBlocks` accept a java lambda.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org