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/02/19 10:24:45 UTC

[GitHub] spark pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

GitHub user jinxing64 opened a pull request:

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

    [SPARK-19659] Fetch big blocks to disk when shuffle-read.

    ## What changes were proposed in this pull request?
    
    Currently the whole block is fetched into memory(off heap by default) when shuffle-read. A block is defined by (shuffleId, mapId, reduceId). Thus it can be large when skew situations. If OOM happens during shuffle read, job will be killed and users will be notified to "Consider boosting spark.yarn.executor.memoryOverhead". Adjusting parameter and allocating more memory can resolve the OOM. However the approach is not perfectly suitable for production environment, especially for data warehouse.
    Using Spark SQL as data engine in warehouse, users hope to have a unified parameter(e.g. memory) but less resource wasted(resource is allocated but not used),
    It's not always easy to predict skew situations, when happen, it make sense to fetch remote blocks to disk for shuffle-read, rather than
    kill the job because of OOM. 
    
    Fetching big blocks to disk is mentioned in SPARK-3019. In this pr, when sendRequest in `ShuffleBlockFetcherIterator`, check if `bytesInFlight` is over `maxBytesInFlight`, if so, fetch remote blocks to disk by sending `StreamRequest`. By setting `spark.reducer.maxSizeInFlight` properly, user can avoid OOM without causing performance degradation.
    
    ## How was this patch tested?
    Not added yet.


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

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

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

    https://github.com/apache/spark/pull/16989.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 #16989
    
----
commit 21f6da36b127956bf35da088f1ecfeb55b307f3e
Author: jinxing <ji...@126.com>
Date:   2017-02-18T15:00:50Z

    [SPARK-19659] Fetch big blocks to disk when shuffle-read.

----


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75938 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75938/testReport)** for PR 16989 at commit [`31cfee0`](https://github.com/apache/spark/commit/31cfee0c8ee51e8fb91276a68a1f2ad35eea410e).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r114965477
  
    --- Diff: core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala ---
    @@ -126,11 +131,21 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext
             .set("spark.shuffle.compress", "false")
             .set("spark.shuffle.spill.compress", "false"))
     
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        override def answer(invocation: InvocationOnMock): Long = {
    +          invocation.getArguments()(0).asInstanceOf[Long]
    +        }
    +      })
    +    taskMemoryManager
    --- End diff --
    
    So sorry.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115639242
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -137,6 +143,9 @@ final class ShuffleBlockFetcherIterator(
         // Release the current buffer if necessary
         if (currentResult != null) {
           currentResult.buf.release()
    +      if (blocksShuffleToMem.contains(currentResult.blockId.toString)) {
    +        freeMemory(currentResult.size)
    --- End diff --
    
    shall we also remove the block id from `blocksShuffleToMem`?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77302 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77302/testReport)** for PR 16989 at commit [`2ce2699`](https://github.com/apache/spark/commit/2ce269991cceaee18fbab71689454c8602342e68).
     * This patch **fails Spark unit 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    In current change:
    1. `ShuffleBlockFetcherIterator` is not a `MemoryConsumer`
    2. Name of shuffle file becomes: ${context.taskAttemptId()}-remote-$bId
    3. Try to delete all shuffle files again in `ShuffleBlockFetcherIterator` `cleanup` again. In current code, `cleanup()` is already registered as a `TaskCompletionListener` in `TaskContext`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115522902
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -137,6 +146,7 @@ final class ShuffleBlockFetcherIterator(
         // Release the current buffer if necessary
         if (currentResult != null) {
           currentResult.buf.release()
    +      freeMemory(currentResult.size)
    --- End diff --
    
    Yes, I should 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75853 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75853/testReport)** for PR 16989 at commit [`63f059d`](https://github.com/apache/spark/commit/63f059de847264f0ecc66bfb83a575e2ca928ae6).


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115525399
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +131,60 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizes sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    @transient var hugeBlockSizes: Map[Int, Byte])
    --- End diff --
    
    @mridulm 
    I removed the `hugeBlockSizesArray` and put `hugeBlockSizes` as transient in constructor. How about this now? I'm not sure how to make this as private. Should I move `hugeBlockSizes` from constructor into Class and create setter for it?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76574 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76574/testReport)** for PR 16989 at commit [`4ad6bda`](https://github.com/apache/spark/commit/4ad6bdaa13755523be3881160d45b17efdedb6f5).
     * This patch **fails SparkR unit 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116904146
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala ---
    @@ -128,4 +138,27 @@ class MapStatusSuite extends SparkFunSuite {
         assert(size1 === size2)
         assert(!success)
       }
    +
    +  test("Blocks which are bigger than spark.shuffle.accurateBlkThreshold should not be " +
    +    "underestimated") {
    +    val conf = new SparkConf().set("spark.shuffle.accurateBlkThreshold", "1000")
    --- End diff --
    
    `xx.set(SHUFFLE_ACCURATE_BLOCK_THRESHOLD, 1000)`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r116911964
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +429,146 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size of the request is above the" +
    +    " threshold(maxReqSizeShuffleToMem.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +          }
    +        }
    +      })
    +
    +    val taskMemoryManager = createMockTaskMemoryManager()
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles === null)
    +
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator2 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress2,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles != null)
    +  }
    +
    +  test("Blocks should be shuffled to disk when size is above memory threshold," +
    +    " otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        // 500 bytes at most can be offered from TaskMemoryManager.
    +        override def answer(invocationOnMock: InvocationOnMock): Long = {
    +          val required = invocationOnMock.getArguments()(0).asInstanceOf[Long]
    +          if (required <= 500) {
    +            return required
    +          } else {
    +            return 500
    --- End diff --
    
    Actually, the blocks are in one request. We will try to allocate 2*300=600.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r124957319
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    --- End diff --
    
    @zsxwing @cloud-fan 
    `OneForOneBlockFetcher` "open blocks" asynchronously. If I understand correctly, the retry of the `start()` in `OneForOneBlockFetcher` is only triggered when failure of sending `OpenBlocks`, but failure of fetching chunk cannot trigger the retry in `RetryingBlockFetcher`. `DownloadCalback` is not initialized when the failure of "open blocks" happens. So there cannot be two `DownloadCallbacks` for same stream working at the same time.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r116907401
  
    --- Diff: docs/configuration.md ---
    @@ -954,16 +970,16 @@ Apart from these, the following properties are also available, and may be useful
       <td><code>spark.memory.offHeap.enabled</code></td>
       <td>false</td>
       <td>
    -    If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then <code>spark.memory.offHeap.size</code> must be positive.
    +    If true, Spark will attempt to use off-heap memory for certain operations(e.g. allocate memory by Unsafe/Tungsten code). If off-heap memory use is enabled, then <code>spark.memory.offHeap.size</code> must be positive.
       </td>
     </tr>
     <tr>
       <td><code>spark.memory.offHeap.size</code></td>
    -  <td>0</td>
    +  <td>384 * 1024 * 1024</td>
       <td>
         The absolute amount of memory in bytes which can be used for off-heap allocation.
         This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly.
    -    This must be set to a positive value when <code>spark.memory.offHeap.enabled=true</code>.
    +    This must be set to a positive value when <code>spark.memory.offHeap.enabled=true</code>. Note that Blocks will be shuffled to off heap by default.
    --- End diff --
    
    Yes, I will fix this. I was thinking Netty will use off-heap for fetching remote blocks when `spark.shuffle.io.preferDirectBufs` is true. That's why I put a `by default` 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76412 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76412/testReport)** for PR 16989 at commit [`1d2b0b9`](https://github.com/apache/spark/commit/1d2b0b915e9956d5231a8fa45710b9efca68529b).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116916099
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +429,146 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size of the request is above the" +
    +    " threshold(maxReqSizeShuffleToMem.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +          }
    +        }
    +      })
    +
    +    val taskMemoryManager = createMockTaskMemoryManager()
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles === null)
    +
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator2 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress2,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles != null)
    +  }
    +
    +  test("Blocks should be shuffled to disk when size is above memory threshold," +
    +    " otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        // 500 bytes at most can be offered from TaskMemoryManager.
    +        override def answer(invocationOnMock: InvocationOnMock): Long = {
    +          val required = invocationOnMock.getArguments()(0).asInstanceOf[Long]
    +          if (required <= 500) {
    +            return required
    +          } else {
    +            return 500
    --- End diff --
    
    ah i see, then let's revert the last change, always return 500 is fine


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75855 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75855/testReport)** for PR 16989 at commit [`63f059d`](https://github.com/apache/spark/commit/63f059de847264f0ecc66bfb83a575e2ca928ae6).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

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


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77243 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77243/testReport)** for PR 16989 at commit [`07fedab`](https://github.com/apache/spark/commit/07fedab0a44a829a2c3fb396f790c59ff494a882).
     * This patch **fails Scala style 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116665589
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -193,8 +216,21 @@ private[spark] object HighlyCompressedMapStatus {
         } else {
           0
         }
    +    val threshold =
    +      SparkEnv.get.conf.getLong("spark.shuffle.accurateBlkThreshold", 100 * 1024 * 1024) max
    +        SparkEnv.get.conf.getInt("spark.shuffle.accurateBlkThresholdByTimesAvg", 2) * avgSize
    --- End diff --
    
    1. let's put the conf in `org.apache.spark.internal.config` object
    2. I think we only need one conf, but not sure which one is better, cc @mridulm 


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117170463
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,50 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    +    }
    +
    +    @Override
    +    public void onComplete(String streamId) throws IOException {
    +      channel.close();
    +      ManagedBuffer buffer = new FileSegmentManagedBuffer(
    +        transportConf, targetFile, 0, targetFile.length()) {
    +        @Override
    +        public ManagedBuffer release() {
    --- End diff --
    
    Is `release()` guaranteed to be called? I noticed that the `targetFile` name is determined by the higher-level code inside of the Spark task itself, so it seems like at that point we know the names of the files which should be cleaned up and could register them with a task completion callback to ensure that they're guaranteed to be deleted after the task finishes. This would provide another layer of defensiveness against disk file leaks.
    
    Aside: this is an example of why we need some TaskContext method for allocating task-scoped temporary file names: it would greatly simplify our story around temporary files like sort spills, etc. Alternatively we could use a temp directory per task which holds all of that task's temporary files and just wipe away the entire directory at the end of the task. This is a larger change that I'm proposing which should probably be done in a separate PR.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117171397
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -278,4 +278,21 @@ package object config {
             "spark.io.compression.codec.")
           .booleanConf
           .createWithDefault(false)
    +
    +  private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
    +    ConfigBuilder("spark.shuffle.accurateBlkThreshold")
    --- End diff --
    
    Excessive abbreviation also makes it harder to grep through the code.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118304568
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -287,4 +287,10 @@ package object config {
           .bytesConf(ByteUnit.BYTE)
           .createWithDefault(100 * 1024 * 1024)
     
    +  private[spark] val REDUCER_MAX_REQ_SIZE_SHUFFLE_TO_MEM =
    +    ConfigBuilder("spark.reducer.maxReqSizeShuffleToMem")
    +      .doc("The blocks of a shuffle request will be fetched to disk when size of the request is " +
    +        "above this threshold. This is to avoid a giant request takes too much memory.")
    +      .bytesConf(ByteUnit.BYTE)
    --- End diff --
    
    the byte unit just indicates the default unit for the value, e.g. `200` means `200m` if the byte unit is mb, `200k` always means `200k` no matter what the byte unit is. And the value we get is always in bytes.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116904336
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala ---
    @@ -51,7 +59,10 @@ private[spark] class BlockStoreShuffleReader[K, C](
           // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility
           SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024,
           SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue),
    -      SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true))
    +      SparkEnv.get.conf.getLong("spark.reducer.maxReqSizeShuffleToMem", 200 * 1024 * 1024),
    --- End diff --
    
    let's also put this to the config object


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77012/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118437769
  
    --- Diff: common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java ---
    @@ -95,6 +97,25 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) {
       }
     
       @Override
    +  public ManagedBuffer openStream(String streamChunkId) {
    +    Tuple2<Long, Integer> streamIdAndChunkId = parseStreamChunkId(streamChunkId);
    +    return getChunk(streamIdAndChunkId._1, streamIdAndChunkId._2);
    +  }
    +
    +  public static String genStreamChunkId(long streamId, int chunkId) {
    +    return String.format("%d_%d", streamId, chunkId);
    +  }
    +
    +  public static Tuple2<Long, Integer> parseStreamChunkId(String streamChunkId) {
    --- End diff --
    
    we can inline it if it becomes a bottleneck 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Jenkins, test 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    In current change:
    1) remove the partial written file when failing
    2) remove all shuffle files when `cleanup()`(this is registered as a task completion callback)


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r114967741
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +130,52 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizesArray sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizesArray: Array[Tuple2[Int, Byte]])
       extends MapStatus with Externalizable {
     
    +  @transient var hugeBlockSizes: Map[Int, Byte] =
    +    if (hugeBlockSizesArray == null) null else hugeBlockSizesArray.toMap
    +
       // loc could be null when the default constructor is called during deserialization
       require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
         "Average size can only be zero for map stages that produced no output")
     
    -  protected def this() = this(null, -1, null, -1)  // For deserialization only
    +  def this() = this(null, -1, null, -1, null)  // For deserialization only
     
       override def location: BlockManagerId = loc
     
       override def getSizeForBlock(reduceId: Int): Long = {
         if (emptyBlocks.contains(reduceId)) {
           0
         } else {
    -      avgSize
    +      hugeBlockSizes.get(reduceId) match {
    +        case Some(size) => MapStatus.decompressSize(size)
    +        case None => avgSize
    +      }
         }
       }
     
       override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
         loc.writeExternal(out)
         emptyBlocks.writeExternal(out)
         out.writeLong(avgSize)
    +    out.writeObject(hugeBlockSizesArray)
       }
     
       override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
         loc = BlockManagerId(in)
         emptyBlocks = new RoaringBitmap()
         emptyBlocks.readExternal(in)
         avgSize = in.readLong()
    +    hugeBlockSizesArray = in.readObject().asInstanceOf[Array[Tuple2[Int, Byte]]]
    +    hugeBlockSizes = hugeBlockSizesArray.toMap
    --- End diff --
    
    I cannot pass "MapStatusSuite: HighlyCompressedMapStatus: estimated size should be the average non-empty block size" if remove this line.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115638115
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +131,60 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizes sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    @transient var hugeBlockSizes: Map[Int, Byte])
    --- End diff --
    
    can't we do `@transient private var hugeBlockSizes: Map[Int, Byte]`?


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75855 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75855/testReport)** for PR 16989 at commit [`63f059d`](https://github.com/apache/spark/commit/63f059de847264f0ecc66bfb83a575e2ca928ae6).
     * This patch **fails Spark unit 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r114696480
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -100,7 +114,14 @@ 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++) {
    -            client.fetchChunk(streamHandle.streamId, i, chunkCallback);
    +            if (fetchToDisk) {
    +              final File targetFile = new File(".",
    --- End diff --
    
    @cloud-fan 
    Yes, but `OneForOneBlockFetcher` is in `network-shuffle` package, I find it hard to import `SparkEnv` from `core` package. Did I miss something?(I'm sorry if this question is stupid.)


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    that seems impossible, can you give an example? BTW if this blocks you, just revert the off-heap config changes.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    LGTM except some 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    @cloud-fan 
    Really really thankful for reviewing this pr:). I've refined according to your comments. Please take another look at this when you have time.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116904487
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala ---
    @@ -51,7 +59,10 @@ private[spark] class BlockStoreShuffleReader[K, C](
           // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility
           SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024,
           SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue),
    -      SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true))
    +      SparkEnv.get.conf.getLong("spark.reducer.maxReqSizeShuffleToMem", 200 * 1024 * 1024),
    --- End diff --
    
    let's also put the off-heap size conf in config object near the `maxReqSizeShuffleToMem` , so that it's easy to see `maxReqSizeShuffleToMem` is smaller than off-heap size by default.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r118272653
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +187,49 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large.
    +    val fetchToDisk = if (req.size > maxReqSizeShuffleToMem) {
    --- End diff --
    
    Sure thing.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76412 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76412/testReport)** for PR 16989 at commit [`1d2b0b9`](https://github.com/apache/spark/commit/1d2b0b915e9956d5231a8fa45710b9efca68529b).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    @cloud-fan Thanks a lot. 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76996 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76996/testReport)** for PR 16989 at commit [`202053d`](https://github.com/apache/spark/commit/202053da201be59c24aca906add89e348249b53a).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117152091
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -278,4 +278,39 @@ package object config {
             "spark.io.compression.codec.")
           .booleanConf
           .createWithDefault(false)
    +
    +  private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
    +    ConfigBuilder("spark.shuffle.accurateBlkThreshold")
    +      .doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will " +
    +        "record the size accurately if it's above the threshold specified by this config. This " +
    +        "helps to prevent OOM by avoiding underestimating shuffle block size when fetch shuffle " +
    +        "blocks.")
    +      .longConf
    +      .createWithDefault(100 * 1024 * 1024)
    +
    +  private[spark] val MEMORY_OFF_HEAP_ENABLED =
    +    ConfigBuilder("spark.memory.offHeap.enabled")
    +      .doc("If true, Spark will attempt to use off-heap memory for certain operations(e.g. sort, " +
    +        "aggregate, etc. However, the buffer used for fetching shuffle blocks is always " +
    +        "off-heap). If off-heap memory use is enabled, then spark.memory.offHeap.size must be " +
    +        "positive.")
    +      .booleanConf
    +      .createWithDefault(false)
    +
    +  private[spark] val MEMORY_OFF_HEAP_SIZE =
    +    ConfigBuilder("spark.memory.offHeap.size")
    +      .doc("The absolute amount of memory in bytes which can be used for off-heap allocation." +
    +        " This setting has no impact on heap memory usage, so if your executors' total memory" +
    +        " consumption must fit within some hard limit then be sure to shrink your JVM heap size" +
    +        " accordingly. This must be set to a positive value when " +
    +        "spark.memory.offHeap.enabled=true.")
    +      .longConf
    --- End diff --
    
    Yes, I should 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 pull request #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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/16989#discussion_r114353841
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala ---
    @@ -42,6 +46,12 @@ private[spark] class BlockStoreShuffleReader[K, C](
     
       /** Read the combined key-values for this reduce task */
       override def read(): Iterator[Product2[K, C]] = {
    +    val memMode =
    +      if (SparkTransportConf.fromSparkConf(SparkEnv.get.conf, "shuffle").preferDirectBufs()) {
    --- End diff --
    
    why don't we look at `spark.memory.offHeap.enabled`?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115884613
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,50 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    +    }
    +
    +    @Override
    +    public void onComplete(String streamId) throws IOException {
    +      channel.close();
    +      ManagedBuffer buffer = new FileSegmentManagedBuffer(
    +        transportConf, targetFile, 0, targetFile.length()) {
    +        @Override
    +        public ManagedBuffer release() {
    +          ManagedBuffer ret = super.release();
    +          if (!targetFile.delete()) {
    +            logger.warn("Failed to cleanup " + targetFile.getAbsolutePath());
    --- End diff --
    
    Yes, I should refine this :-)


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Jenkins, add to whitelist


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77238 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77238/testReport)** for PR 16989 at commit [`63b3292`](https://github.com/apache/spark/commit/63b3292abb714bd688a80665164ec4d84b994821).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76995 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76995/testReport)** for PR 16989 at commit [`f353302`](https://github.com/apache/spark/commit/f3533022ab170b82513ac2d9a8e977a3db0a260d).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115410136
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +130,52 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizesArray sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizesArray: Array[Tuple2[Int, Byte]])
       extends MapStatus with Externalizable {
     
    +  @transient var hugeBlockSizes: Map[Int, Byte] =
    +    if (hugeBlockSizesArray == null) null else hugeBlockSizesArray.toMap
    +
       // loc could be null when the default constructor is called during deserialization
       require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
         "Average size can only be zero for map stages that produced no output")
     
    -  protected def this() = this(null, -1, null, -1)  // For deserialization only
    +  def this() = this(null, -1, null, -1, null)  // For deserialization only
     
       override def location: BlockManagerId = loc
     
       override def getSizeForBlock(reduceId: Int): Long = {
         if (emptyBlocks.contains(reduceId)) {
           0
         } else {
    -      avgSize
    +      hugeBlockSizes.get(reduceId) match {
    +        case Some(size) => MapStatus.decompressSize(size)
    +        case None => avgSize
    +      }
         }
       }
     
       override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
         loc.writeExternal(out)
         emptyBlocks.writeExternal(out)
         out.writeLong(avgSize)
    +    out.writeObject(hugeBlockSizesArray)
       }
     
       override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
         loc = BlockManagerId(in)
         emptyBlocks = new RoaringBitmap()
         emptyBlocks.readExternal(in)
         avgSize = in.readLong()
    +    hugeBlockSizesArray = in.readObject().asInstanceOf[Array[Tuple2[Int, Byte]]]
    +    hugeBlockSizes = hugeBlockSizesArray.toMap
    --- End diff --
    
    Object creation (this()) has already happened - readExternal is restoring the state from the stream. So we need to keep this @cloud-fan 
    
    One issue I have here is that we are duplicating the information between hugeBlockSizesArray and hugeBlockSizes.
    I would prefer if we dropped hugeBlockSizesArray entirely (other than as constructor param we initialize state from).
    This will actually result in more efficient serde at the cost of manually doing the serde for hugeBlockSizes, and handle all the corner cases (like avoid need for any null check, etc).
    For serialization: write length, loop - write key as int, write value as byte; for deserialization, the reverse.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117173550
  
    --- Diff: core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala ---
    @@ -29,7 +29,11 @@ import org.apache.spark.shuffle.FetchFailedException
     import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId}
     
     class MapOutputTrackerSuite extends SparkFunSuite {
    +
       private val conf = new SparkConf
    +  val env = mock(classOf[SparkEnv])
    --- End diff --
    
    It's a little weird that we have to modify this. I kind of understand why we don't want to thread SparkEnv or another setting into the `MapStatus` constructor but maybe we could avoid the confusing NPEs that this prevents by doing something like
    
    ```
    Option(SparkEnv.get).map(_.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD)).getOrElse(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.default)
    ```
    
    thereby minimizing changes to other files / tests.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    @squito 
    Thanks a lot for your comments : )
    Yes, There must be a design doc for discussing. I will prepare and  post a pdf to jira.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116476752
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -63,8 +67,12 @@ final class ShuffleBlockFetcherIterator(
         streamWrapper: (BlockId, InputStream) => InputStream,
         maxBytesInFlight: Long,
         maxReqsInFlight: Int,
    -    detectCorrupt: Boolean)
    -  extends Iterator[(BlockId, InputStream)] with Logging {
    +    maxReqSizeShuffleToMem: Int,
    +    detectCorrupt: Boolean,
    +    tmm: TaskMemoryManager,
    +    memMode: MemoryMode = MemoryMode.OFF_HEAP)
    +      extends MemoryConsumer(tmm, tmm.pageSizeBytes(), memMode)
    --- End diff --
    
    the indention is wrong 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115523103
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -154,15 +164,24 @@ final class ShuffleBlockFetcherIterator(
         while (iter.hasNext) {
           val result = iter.next()
           result match {
    -        case SuccessFetchResult(_, address, _, buf, _) =>
    +        case SuccessFetchResult(_, address, size, buf, _) =>
               if (address != blockManager.blockManagerId) {
                 shuffleMetrics.incRemoteBytesRead(buf.size)
                 shuffleMetrics.incRemoteBlocksFetched(1)
               }
               buf.release()
    +          freeMemory(size)
             case _ =>
           }
         }
    +    shuffleFiles.foreach { shuffleFile =>
    +      try {
    +        shuffleFile.delete()
    +      } catch {
    +        case ioe: IOException =>
    +          logError(s"Failed to cleanup ${shuffleFile.getAbsolutePath}.", ioe)
    --- End diff --
    
    Yes, sorry for the mistake


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76483/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Checking the code:
    https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala#L59
    `SparkConfigProvider` just check if the key is in JMap, if not return the default value. It doesn't check the alternatives.
    I think it seems this is the reason  `org.apache.spark.memory.TaskMemoryManagerSuite.offHeapConfigurationBackwardsCompatibility ` fails.


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115412242
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -137,6 +146,7 @@ final class ShuffleBlockFetcherIterator(
         // Release the current buffer if necessary
         if (currentResult != null) {
           currentResult.buf.release()
    +      freeMemory(currentResult.size)
    --- End diff --
    
    Only if in memory and not on 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 pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115825023
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,50 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    +    }
    +
    +    @Override
    +    public void onComplete(String streamId) throws IOException {
    +      channel.close();
    +      ManagedBuffer buffer = new FileSegmentManagedBuffer(
    +        transportConf, targetFile, 0, targetFile.length()) {
    +        @Override
    +        public ManagedBuffer release() {
    +          ManagedBuffer ret = super.release();
    +          if (!targetFile.delete()) {
    +            logger.warn("Failed to cleanup " + targetFile.getAbsolutePath());
    --- End diff --
    
    Perhaps WARN is too strong here, INFO or lower should do.
    It is possible for files to be cleaned up as part of shutdown (for example) and we dont want lot of warn messages in log


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76812 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76812/testReport)** for PR 16989 at commit [`5ad4c3e`](https://github.com/apache/spark/commit/5ad4c3e5ce84a4f82b83517bf3ff92b78f3f98be).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77006/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r117181192
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -193,8 +217,19 @@ private[spark] object HighlyCompressedMapStatus {
         } else {
           0
         }
    +    val threshold = SparkEnv.get.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD)
    +    val hugeBlockSizesArray = ArrayBuffer[Tuple2[Int, Byte]]()
    --- 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 pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117171649
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -278,4 +278,21 @@ package object config {
             "spark.io.compression.codec.")
           .booleanConf
           .createWithDefault(false)
    +
    +  private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
    +    ConfigBuilder("spark.shuffle.accurateBlkThreshold")
    +      .doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will " +
    +        "record the size accurately if it's above the threshold specified by this config. This " +
    --- End diff --
    
    One edge-case to consider is the situation where every shuffle block is _just_ over this threshold: in this case `HighlyCompressedMapStatus` won't really be doing any compression.
    
    Does it make sense to compare to the average and capture the sizes of blocks which are more than some percent / threshold above the average? The number of such blocks will probably be smaller and this might help to avoid worst-case behaviors or excessive bloating of the map output status sizes were someone to set this configuration too low.


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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/16989#discussion_r114355876
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala ---
    @@ -42,6 +46,12 @@ private[spark] class BlockStoreShuffleReader[K, C](
     
       /** Read the combined key-values for this reduce task */
       override def read(): Iterator[Product2[K, C]] = {
    +    val memMode =
    --- End diff --
    
    can we move this into `ShuffleBlockFetcherIterator`?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77006 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77006/testReport)** for PR 16989 at commit [`1ed5eb6`](https://github.com/apache/spark/commit/1ed5eb6171e658d67f0cd5310d805c6ece70d86d).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r118721863
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +188,45 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large.
    +    // TODO: Encryption and compression should be considered.
    --- End diff --
    
    >  do we need to encrypt and compress the data for sort buffer spill and aggregate buffer spill? 
    
    Yes, but I thought I had done that in a previous change. Maybe I missed something.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r114965988
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +424,74 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size is above the threshold, otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFilesOpt: Option[Array[File]] = None
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFilesOpt = invocation.getArguments()(5).asInstanceOf[Option[Array[File]]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        override def answer(invocationOnMock: InvocationOnMock): Long = 500L
    +      })
    +
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100.asInstanceOf[Long])).toSeq))
    +
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      48 * 1024 * 1024,
    +      Int.MaxValue,
    +      true,
    +      taskMemoryManager)
    +
    +    assert(shuffleFilesOpt.isEmpty)
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 200.asInstanceOf[Long])).toSeq)
    --- End diff --
    
    oh...


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118621216
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +188,45 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large.
    +    // TODO: Encryption and compression should be considered.
    --- End diff --
    
    Actually I'm just going to send a follow-up PR. Ideally all local files written by Spark could be encrypted and compressed according to config. One example is `UnsafeSorterSpillWriter`, it writes data with `DiskBlockObjectWriter`, which calls `SerializerManager.wrapStream` and handles encryption and compression automatically.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77302/
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77316 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77316/testReport)** for PR 16989 at commit [`3971706`](https://github.com/apache/spark/commit/39717065c2b6fc6fe6eb4c401b7fb27f72bda326).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r117151567
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -278,4 +278,39 @@ package object config {
             "spark.io.compression.codec.")
           .booleanConf
           .createWithDefault(false)
    +
    +  private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
    +    ConfigBuilder("spark.shuffle.accurateBlkThreshold")
    +      .doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will " +
    +        "record the size accurately if it's above the threshold specified by this config. This " +
    +        "helps to prevent OOM by avoiding underestimating shuffle block size when fetch shuffle " +
    +        "blocks.")
    +      .longConf
    +      .createWithDefault(100 * 1024 * 1024)
    +
    +  private[spark] val MEMORY_OFF_HEAP_ENABLED =
    +    ConfigBuilder("spark.memory.offHeap.enabled")
    +      .doc("If true, Spark will attempt to use off-heap memory for certain operations(e.g. sort, " +
    +        "aggregate, etc. However, the buffer used for fetching shuffle blocks is always " +
    +        "off-heap). If off-heap memory use is enabled, then spark.memory.offHeap.size must be " +
    +        "positive.")
    +      .booleanConf
    +      .createWithDefault(false)
    +
    +  private[spark] val MEMORY_OFF_HEAP_SIZE =
    +    ConfigBuilder("spark.memory.offHeap.size")
    +      .doc("The absolute amount of memory in bytes which can be used for off-heap allocation." +
    +        " This setting has no impact on heap memory usage, so if your executors' total memory" +
    +        " consumption must fit within some hard limit then be sure to shrink your JVM heap size" +
    +        " accordingly. This must be set to a positive value when " +
    +        "spark.memory.offHeap.enabled=true.")
    +      .longConf
    --- End diff --
    
    we should use `.bytesConf(ByteUnit.BYTE)`, see `SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE` as an example


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76630/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r124908306
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    --- End diff --
    
    Does this work with `RetryingBlockFetcher`? Let's say we have 2 chunks: "chunk 1", "chunk 2". If "chunk 1" fails, it will fail "chunk 2" as well. However, DownloadCallbacks for "chunk 2" are still running. In this case, RetryingBlockFetcher will retry "chunk 2" as well. Hence, there will be 2 DownloadCallbacks writing to the same file.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76996/
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    @squito 
    I've uploaded a design doc to jira, please take a look when you have time :)


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117172062
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -395,7 +438,6 @@ final class ShuffleBlockFetcherIterator(
           // Send fetch requests up to maxBytesInFlight
           fetchUpToMaxBytes()
         }
    -
    --- End diff --
    
    Remove unnecessary whitespace change?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76997/
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76489/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77039 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77039/testReport)** for PR 16989 at commit [`4ece142`](https://github.com/apache/spark/commit/4ece142d2a3c4b46a712539e3aa7f7ee0d4e6b5b).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117203833
  
    --- Diff: common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java ---
    @@ -95,6 +97,25 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) {
       }
     
       @Override
    +  public ManagedBuffer openStream(String streamChunkId) {
    +    Tuple2<Long, Integer> streamIdAndChunkId = parseStreamChunkId(streamChunkId);
    +    return getChunk(streamIdAndChunkId._1, streamIdAndChunkId._2);
    +  }
    +
    +  public static String genStreamChunkId(long streamId, int chunkId) {
    +    return String.format("%d_%d", streamId, chunkId);
    +  }
    +
    +  public static Tuple2<Long, Integer> parseStreamChunkId(String streamChunkId) {
    --- End diff --
    
    Idea was to ensure that string parsing and unparsing it localized to a single place (genStreamChunkId and parseStreamChunkId) - so that we can modify it in future if required without needing to look for where all we did a split by "_"


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r124959257
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    --- End diff --
    
    @jinxing64 The retry logic is here: https://github.com/apache/spark/blob/88a536babf119b7e331d02aac5d52b57658803bf/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java#L215
    
    The issue is there will be two `DownloadCallback`s download the same content to the same target file. While the first one finishes, ShuffleBlockFetcherIterator may start to read it, however, the second `DownloadCallback` may be still running and writing to the target file. It could cause `ShuffleBlockFetcherIterator` reading a partial result.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Can one of the admins verify this patch?


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75358 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75358/testReport)** for PR 16989 at commit [`822b125`](https://github.com/apache/spark/commit/822b125b328680ed009e9819e8c4d9b812094a9e).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r116521474
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala ---
    @@ -51,7 +59,10 @@ private[spark] class BlockStoreShuffleReader[K, C](
           // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility
           SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024,
           SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue),
    -      SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true))
    +      SparkEnv.get.conf.getLong("spark.reducer.maxReqSizeShuffleToMem", 200 * 1024 * 1024),
    --- End diff --
    
    set this lower than spark.memory.offHeap.size


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r114949186
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +151,39 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    +    }
    +
    +    @Override
    +    public void onComplete(String streamId) throws IOException {
    +      channel.close();
    +      ManagedBuffer buffer = new FileSegmentManagedBuffer(
    --- End diff --
    
    a question: when shall we remove the file after the data is all consumed?


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r116674324
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -193,8 +216,21 @@ private[spark] object HighlyCompressedMapStatus {
         } else {
           0
         }
    +    val threshold =
    +      SparkEnv.get.conf.getLong("spark.shuffle.accurateBlkThreshold", 100 * 1024 * 1024) max
    +        SparkEnv.get.conf.getInt("spark.shuffle.accurateBlkThresholdByTimesAvg", 2) * avgSize
    --- End diff --
    
    1. Yes, I will put the conf in `org.apache.spark.internal.config`.
    2. How about my previous change, `SparkEnv.get.conf.getLong("spark.shuffle.accurateBlkThreshold", 2 * avgSize)` ?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76997 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76997/testReport)** for PR 16989 at commit [`f164cd6`](https://github.com/apache/spark/commit/f164cd6998152fe71c1177599a9071beb3404751).
     * 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75441 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75441/testReport)** for PR 16989 at commit [`1ec1c0b`](https://github.com/apache/spark/commit/1ec1c0bf742d8efb6ffc7a37c001a3f74961c473).
     * 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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

    https://github.com/apache/spark/pull/16989#discussion_r114504511
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala ---
    @@ -42,6 +46,12 @@ private[spark] class BlockStoreShuffleReader[K, C](
     
       /** Read the combined key-values for this reduce task */
       override def read(): Iterator[Product2[K, C]] = {
    +    val memMode =
    --- End diff --
    
    Yes, ideally this should be moved into `ShuffleBlockFetcherIterator`, but I didn't find a better implementation other than 
    ```
          extends MemoryConsumer(tmm, tmm.pageSizeBytes(),
            if (SparkTransportConf.fromSparkConf(SparkEnv.get.conf, "shuffle").preferDirectBufs()) {
              MemoryMode.OFF_HEAP
            } else {
              MemoryMode.ON_HEAP
            }
          )
    ```
    And I'd be a little bit hesitant to expose a 'setMode' in `MemoryConsumer`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76925 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76925/testReport)** for PR 16989 at commit [`80b3154`](https://github.com/apache/spark/commit/80b31545a1d6b6890e3cc0d549781ca15d7d46dc).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115410895
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -193,8 +206,18 @@ private[spark] object HighlyCompressedMapStatus {
         } else {
           0
         }
    +    val hugeBlockSizes = ArrayBuffer[Tuple2[Int, Byte]]()
    +    if (numNonEmptyBlocks > 0) {
    +      uncompressedSizes.zipWithIndex.foreach {
    +        case (size, reduceId) =>
    +          if (size > 2 * avgSize) {
    --- End diff --
    
    This should be configurable in two respects.
    * minimum size before we consider something a large block : if average is 10kb, and some blocks are > 20kb, spilling them to disk would be highly suboptimal. (Unless I missed that check somewhere else).
    * The fraction '2' should also be configurable - some deployments might be ok with high memory usage (machines provisioned accordingly) while others might need it to be more aggressive and lower.
    



---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77302 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77302/testReport)** for PR 16989 at commit [`2ce2699`](https://github.com/apache/spark/commit/2ce269991cceaee18fbab71689454c8602342e68).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116927503
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +429,146 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size of the request is above the" +
    +    " threshold(maxReqSizeShuffleToMem).") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +          }
    +        }
    +      })
    +
    +    val taskMemoryManager = createMockTaskMemoryManager()
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles === null)
    +
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator2 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress2,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles != null)
    +  }
    +
    +  test("Blocks should be shuffled to disk when size is above memory threshold," +
    +    " otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        // 500 bytes at most can be offered from TaskMemoryManager.
    +        override def answer(invocationOnMock: InvocationOnMock): Long = {
    +          val required = invocationOnMock.getArguments()(0).asInstanceOf[Long]
    +          if (required <= 500) {
    +            return required
    --- End diff --
    
    nit:
    ```
    val required = ...
    math.min(required, 500)
    ```


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r124957411
  
    --- Diff: common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java ---
    @@ -95,6 +97,25 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) {
       }
     
       @Override
    +  public ManagedBuffer openStream(String streamChunkId) {
    --- End diff --
    
    Thanks, I will try make a pr as soon as possible.


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75858 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75858/testReport)** for PR 16989 at commit [`b6a8993`](https://github.com/apache/spark/commit/b6a8993ee96da8be336603e3699e29582c157130).
     * This patch **fails Scala style 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117205121
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +197,54 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large or local memory shortage.
    +    val fetchToDisk = if (req.size > maxReqSizeShuffleToMem) {
    +      true
    +    } else {
    +      val acquired = acquireMemory(req.size)
    +      if (acquired < req.size) {
    +        freeMemory(acquired)
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    if (fetchToDisk) {
    +      shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    +        blockFetchingListener,
    +        blockIds.map(bId => blockManager.diskBlockManager.getFile(s"remote-$bId")).toArray)
    --- End diff --
    
    This is a good point, and it does actually happen today for custom joins.
    I missed that the file name was not random, nice catch !


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r118424377
  
    --- Diff: docs/configuration.md ---
    @@ -520,6 +520,14 @@ Apart from these, the following properties are also available, and may be useful
       </td>
     </tr>
     <tr>
    +  <td><code>spark.reducer.maxReqSizeShuffleToMem</code></td>
    +  <td>200 * 1024 * 1024</td>
    --- End diff --
    
    Yes, I should fix this


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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/16989#discussion_r114352884
  
    --- Diff: core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala ---
    @@ -388,7 +388,8 @@ private[serializer] object KryoSerializer {
         classOf[Array[Short]],
         classOf[Array[Long]],
         classOf[BoundedPriorityQueue[_]],
    -    classOf[SparkConf]
    +    classOf[SparkConf],
    +    classOf[HashMap[Int, Byte]]
    --- End diff --
    
    how is the overhead when serializing hash map with kryo?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115824704
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +149,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    --- End diff --
    
    I was just noting it for review :-) in case someone else goes through the same concern.
    We are relying on impl detail of what the `channel` is here to avoid checking if bug.remaining == 0 after `channel.write` returns.
    StreamCallback expects it to be empty after onData is done.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r114960041
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +130,52 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizesArray sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizesArray: Array[Tuple2[Int, Byte]])
       extends MapStatus with Externalizable {
     
    +  @transient var hugeBlockSizes: Map[Int, Byte] =
    +    if (hugeBlockSizesArray == null) null else hugeBlockSizesArray.toMap
    +
       // loc could be null when the default constructor is called during deserialization
       require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
         "Average size can only be zero for map stages that produced no output")
     
    -  protected def this() = this(null, -1, null, -1)  // For deserialization only
    +  def this() = this(null, -1, null, -1, null)  // For deserialization only
     
       override def location: BlockManagerId = loc
     
       override def getSizeForBlock(reduceId: Int): Long = {
         if (emptyBlocks.contains(reduceId)) {
           0
         } else {
    -      avgSize
    +      hugeBlockSizes.get(reduceId) match {
    +        case Some(size) => MapStatus.decompressSize(size)
    +        case None => avgSize
    +      }
         }
       }
     
       override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
         loc.writeExternal(out)
         emptyBlocks.writeExternal(out)
         out.writeLong(avgSize)
    +    out.writeObject(hugeBlockSizesArray)
       }
     
       override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
         loc = BlockManagerId(in)
         emptyBlocks = new RoaringBitmap()
         emptyBlocks.readExternal(in)
         avgSize = in.readLong()
    +    hugeBlockSizesArray = in.readObject().asInstanceOf[Array[Tuple2[Int, Byte]]]
    +    hugeBlockSizes = hugeBlockSizesArray.toMap
    --- End diff --
    
    After the constructor is called, both `hugeBlockSizes` and `hugeBlockSizesArray` are initialized to be null, I think we need to initialize both in `readExternal` when deserialize.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76995/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r116919856
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +429,146 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size of the request is above the" +
    +    " threshold(maxReqSizeShuffleToMem.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +          }
    +        }
    +      })
    +
    +    val taskMemoryManager = createMockTaskMemoryManager()
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles === null)
    +
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator2 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress2,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles != null)
    +  }
    +
    +  test("Blocks should be shuffled to disk when size is above memory threshold," +
    +    " otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        // 500 bytes at most can be offered from TaskMemoryManager.
    +        override def answer(invocationOnMock: InvocationOnMock): Long = {
    +          val required = invocationOnMock.getArguments()(0).asInstanceOf[Long]
    +          if (required <= 500) {
    +            return required
    +          } else {
    +            return 500
    --- End diff --
    
    Well, current change is a also correct(more accurate?). Say we require 300, 300 are allocated, not always 500.


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    overall LGTM, left some minor comments, thanks for working on it!


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    #18031 has been merged, can you update? thanks!


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77321/
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115638943
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +131,60 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizes sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    @transient var hugeBlockSizes: Map[Int, Byte])
    --- End diff --
    
    I tried but failed to compile:(


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    hey don't forget this comment :) https://github.com/apache/spark/pull/16989/files#r118183414


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118304766
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -154,7 +161,7 @@ final class ShuffleBlockFetcherIterator(
         while (iter.hasNext) {
           val result = iter.next()
           result match {
    -        case SuccessFetchResult(_, address, _, buf, _) =>
    +        case SuccessFetchResult(bId, address, _, buf, _) =>
    --- End diff --
    
    why this change?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116478360
  
    --- Diff: docs/configuration.md ---
    @@ -512,6 +512,14 @@ Apart from these, the following properties are also available, and may be useful
       </td>
     </tr>
     <tr>
    +  <td><code>spark.reducer.maxReqSizeShuffleToMem</code></td>
    +  <td>Int.MaxValue</td>
    --- End diff --
    
    shall we use a smaller default value?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76630 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76630/testReport)** for PR 16989 at commit [`308b7c7`](https://github.com/apache/spark/commit/308b7c72984d66030551f58ba000c5090d308dde).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76574 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76574/testReport)** for PR 16989 at commit [`4ad6bda`](https://github.com/apache/spark/commit/4ad6bdaa13755523be3881160d45b17efdedb6f5).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118637335
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +188,45 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large.
    +    // TODO: Encryption and compression should be considered.
    --- End diff --
    
    ah that's a good point! Yea we don't need to encrypt and compress the data again here. I'll update this comment.
    
    One question: do we need to encrypt and compress the data for sort buffer spill and aggregate buffer spill? cc @JoshRosen 


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r114954303
  
    --- Diff: core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala ---
    @@ -126,11 +131,21 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext
             .set("spark.shuffle.compress", "false")
             .set("spark.shuffle.spill.compress", "false"))
     
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        override def answer(invocation: InvocationOnMock): Long = {
    +          invocation.getArguments()(0).asInstanceOf[Long]
    +        }
    +      })
    +    taskMemoryManager
    --- End diff --
    
    why this line?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115409268
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +151,39 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    +    }
    +
    +    @Override
    +    public void onComplete(String streamId) throws IOException {
    +      channel.close();
    +      ManagedBuffer buffer = new FileSegmentManagedBuffer(
    --- End diff --
    
    After consumption of each corresponding ManagedBuffer, we should make an attempt to remove the corresponding file : should be fairly straightforward, no ? (override release ?)



---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    LGTM except for one comment


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115410407
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +130,52 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizesArray sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizesArray: Array[Tuple2[Int, Byte]])
       extends MapStatus with Externalizable {
     
    +  @transient var hugeBlockSizes: Map[Int, Byte] =
    +    if (hugeBlockSizesArray == null) null else hugeBlockSizesArray.toMap
    +
       // loc could be null when the default constructor is called during deserialization
       require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
         "Average size can only be zero for map stages that produced no output")
     
    -  protected def this() = this(null, -1, null, -1)  // For deserialization only
    +  protected def this() = this(null, -1, null, -1, null)  // For deserialization only
     
       override def location: BlockManagerId = loc
     
       override def getSizeForBlock(reduceId: Int): Long = {
         if (emptyBlocks.contains(reduceId)) {
           0
         } else {
    -      avgSize
    +      hugeBlockSizes.get(reduceId) match {
    --- End diff --
    
    NPE


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75441 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75441/testReport)** for PR 16989 at commit [`1ec1c0b`](https://github.com/apache/spark/commit/1ec1c0bf742d8efb6ffc7a37c001a3f74961c473).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r118630394
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +188,45 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large.
    +    // TODO: Encryption and compression should be considered.
    --- End diff --
    
    I haven't really followed this review (sorry), but shuffle data is transmitted encrypted and compressed over the wire, so there might be a chance that there's nothing to do 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    last few minor comments, I think we are ready to go :)


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77246 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77246/testReport)** for PR 16989 at commit [`ac12325`](https://github.com/apache/spark/commit/ac12325a408e248d9b52b6cecb551454fd6c48b5).
     * 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115012638
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala ---
    @@ -128,4 +130,23 @@ class MapStatusSuite extends SparkFunSuite {
         assert(size1 === size2)
         assert(!success)
       }
    +
    +  test("Blocks which are bigger than 2 * average size should not be underestimated.") {
    +    val sizes = Array.concat(Array.fill[Long](1000)(1L), (1000L to 2000L).toArray)
    +    val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes)
    +    val arrayStream = new ByteArrayOutputStream(102400)
    +    val objectOutputStream = new ObjectOutputStream(arrayStream)
    +    assert(status1.isInstanceOf[HighlyCompressedMapStatus])
    +    status1.asInstanceOf[HighlyCompressedMapStatus].writeExternal(objectOutputStream)
    --- End diff --
    
    I see the problem. This is not the normal process for serializing java objects, `writeExternal`/`readExternal` should be called by java serialization framework, instead of being called manually. We should do `objectOutputStream.writeObject(status1)`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77247 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77247/testReport)** for PR 16989 at commit [`ac030fa`](https://github.com/apache/spark/commit/ac030fa08203bf6dbdcaa21aa5dc8b86389a3e16).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76485/
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75436 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75436/testReport)** for PR 16989 at commit [`65e7c42`](https://github.com/apache/spark/commit/65e7c42c00369eec6f1138848c3af0b0788f7937).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76812/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    In current code, `spark.memory.offHeap.enabled` is used when decide `tungstenMemoryMode`.
    `spark.memory.offHeap.enabled` doesn't decide remote blocks are shuffled to whether onHeap or offHeap.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r116907421
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +429,139 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size of the request is above the" +
    +    " threshold(maxReqSizeShuffleToMem.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +          }
    +        }
    +      })
    +
    +    val taskMemoryManager = createMockTaskMemoryManager()
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles === null)
    +
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator2 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress2,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles != null)
    +  }
    +
    +  test("Blocks should be shuffled to disk when size is above memory threshold," +
    +    " otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        // 500 bytes at most can be offered from TaskMemoryManager.
    +        override def answer(invocationOnMock: InvocationOnMock): Long = 500L
    --- End diff --
    
    Yes, I will fix this.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #73224 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73224/testReport)** for PR 16989 at commit [`21f6da3`](https://github.com/apache/spark/commit/21f6da36b127956bf35da088f1ecfeb55b307f3e).


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77237 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77237/testReport)** for PR 16989 at commit [`283746b`](https://github.com/apache/spark/commit/283746bbbb169a8873d8608dbb2507761f347fdf).


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75358/
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76414 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76414/testReport)** for PR 16989 at commit [`435573a`](https://github.com/apache/spark/commit/435573a911c16c63c24aced24ea9c2902a043103).


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75435/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76497/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77247/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118188723
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +187,49 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large.
    +    val fetchToDisk = if (req.size > maxReqSizeShuffleToMem) {
    +      true
    +    } else {
    +      false
    +    }
    +
    +    if (fetchToDisk) {
    +      val shuffleFiles = blockIds.map(bId => blockManager.diskBlockManager
    +          .getFile(s"${context.taskAttemptId()}-remote-$bId")).toArray
    --- End diff --
    
    we should also consider encryption and compression here, let's add a TODO


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115827377
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -193,8 +215,18 @@ private[spark] object HighlyCompressedMapStatus {
         } else {
           0
         }
    +    val hugeBlockSizesArray = ArrayBuffer[Tuple2[Int, Byte]]()
    +    if (numNonEmptyBlocks > 0) {
    +      uncompressedSizes.zipWithIndex.foreach {
    +        case (size, reduceId) =>
    +          if (size > 2 * avgSize) {
    --- End diff --
    
    This is yet to be addressed - the scale constant `2`, and putting a lower bound on `size`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r114954964
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +424,74 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size is above the threshold, otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFilesOpt: Option[Array[File]] = None
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFilesOpt = invocation.getArguments()(5).asInstanceOf[Option[Array[File]]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        override def answer(invocationOnMock: InvocationOnMock): Long = 500L
    +      })
    +
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100.asInstanceOf[Long])).toSeq))
    +
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      48 * 1024 * 1024,
    +      Int.MaxValue,
    +      true,
    +      taskMemoryManager)
    +
    +    assert(shuffleFilesOpt.isEmpty)
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 200.asInstanceOf[Long])).toSeq)
    --- End diff --
    
    same 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75853 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75853/testReport)** for PR 16989 at commit [`63f059d`](https://github.com/apache/spark/commit/63f059de847264f0ecc66bfb83a575e2ca928ae6).
     * This patch **fails Spark unit 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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

    https://github.com/apache/spark/pull/16989#discussion_r114503489
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -100,7 +114,14 @@ 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++) {
    -            client.fetchChunk(streamHandle.streamId, i, chunkCallback);
    +            if (fetchToDisk) {
    +              final File targetFile = new File(".",
    --- End diff --
    
    Yes, I wanted to use `DiskBlockManager.getFile`, but I found it's hard to import `DiskBlockManager` from `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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76943/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115409386
  
    --- Diff: core/src/main/scala/org/apache/spark/memory/MemoryManager.scala ---
    @@ -54,7 +54,8 @@ private[spark] abstract class MemoryManager(
       onHeapStorageMemoryPool.incrementPoolSize(onHeapStorageMemory)
       onHeapExecutionMemoryPool.incrementPoolSize(onHeapExecutionMemory)
     
    -  protected[this] val maxOffHeapMemory = conf.getSizeAsBytes("spark.memory.offHeap.size", 0)
    +  protected[this] val maxOffHeapMemory =
    +    conf.getSizeAsBytes("spark.memory.offHeap.size", 384 * 1024 * 1024)
    --- End diff --
    
    Maybe I missed the discussion, why is this changed ?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76808 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76808/testReport)** for PR 16989 at commit [`778c59b`](https://github.com/apache/spark/commit/778c59bcc6dbf98d410a6fe3718594a6ecafbfbd).
     * This patch **fails Spark unit 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118186552
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +187,49 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large.
    +    val fetchToDisk = if (req.size > maxReqSizeShuffleToMem) {
    --- End diff --
    
    we can inline this condition


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77243/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Jenkins, test 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75855/
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75853/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115217559
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +130,52 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizesArray sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizesArray: Array[Tuple2[Int, Byte]])
       extends MapStatus with Externalizable {
     
    +  @transient var hugeBlockSizes: Map[Int, Byte] =
    +    if (hugeBlockSizesArray == null) null else hugeBlockSizesArray.toMap
    +
       // loc could be null when the default constructor is called during deserialization
       require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
         "Average size can only be zero for map stages that produced no output")
     
    -  protected def this() = this(null, -1, null, -1)  // For deserialization only
    +  def this() = this(null, -1, null, -1, null)  // For deserialization only
     
       override def location: BlockManagerId = loc
     
       override def getSizeForBlock(reduceId: Int): Long = {
         if (emptyBlocks.contains(reduceId)) {
           0
         } else {
    -      avgSize
    +      hugeBlockSizes.get(reduceId) match {
    +        case Some(size) => MapStatus.decompressSize(size)
    +        case None => avgSize
    +      }
         }
       }
     
       override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
         loc.writeExternal(out)
         emptyBlocks.writeExternal(out)
         out.writeLong(avgSize)
    +    out.writeObject(hugeBlockSizesArray)
       }
     
       override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
         loc = BlockManagerId(in)
         emptyBlocks = new RoaringBitmap()
         emptyBlocks.readExternal(in)
         avgSize = in.readLong()
    +    hugeBlockSizesArray = in.readObject().asInstanceOf[Array[Tuple2[Int, Byte]]]
    +    hugeBlockSizes = hugeBlockSizesArray.toMap
    --- End diff --
    
    do we still need this after fixing the `MapStatusSuite`?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #73224 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73224/testReport)** for PR 16989 at commit [`21f6da3`](https://github.com/apache/spark/commit/21f6da36b127956bf35da088f1ecfeb55b307f3e).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76415 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76415/testReport)** for PR 16989 at commit [`4045d6e`](https://github.com/apache/spark/commit/4045d6ea25bccdd347aeed7c3b042086822d587c).
     * 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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

    https://github.com/apache/spark/pull/16989#discussion_r114503627
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -133,36 +135,53 @@ private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizes: HashMap[Int, Byte])
    --- End diff --
    
    Yes, 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73224/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115218468
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -163,6 +173,8 @@ final class ShuffleBlockFetcherIterator(
             case _ =>
           }
         }
    +    freeMemory(getUsed)
    +    shuffleFiles.foreach(_.delete())
    --- End diff --
    
    Since these are just temp files, we should not fail the job if file deletion failed. Let's try catch the deletion and log the exception instead of throwing it.


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

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


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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/16989#discussion_r114351675
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -133,36 +135,53 @@ private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizes: HashMap[Int, Byte])
    --- End diff --
    
    add parameter doc for this


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76944/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76486 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76486/testReport)** for PR 16989 at commit [`4b4e42e`](https://github.com/apache/spark/commit/4b4e42eb0cc340548fe4af0177736610aa59ec9f).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r125877439
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    --- End diff --
    
    @zsxwing 
    Sorry, I just realized this issue. There can be conflict between two `DownloadCallback`s. I will figure out a way to resolve this.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115220025
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -163,6 +173,8 @@ final class ShuffleBlockFetcherIterator(
             case _ =>
           }
         }
    +    freeMemory(getUsed)
    --- End diff --
    
    Yes, I should 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r116830282
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -193,8 +216,21 @@ private[spark] object HighlyCompressedMapStatus {
         } else {
           0
         }
    +    val threshold =
    +      SparkEnv.get.conf.getLong("spark.shuffle.accurateBlkThreshold", 100 * 1024 * 1024) max
    +        SparkEnv.get.conf.getInt("spark.shuffle.accurateBlkThresholdByTimesAvg", 2) * avgSize
    --- End diff --
    
    Agree about moving it to `org.apache.spark.internal.config`.
    
    I am fine with it either way - a constant size does have the potential for degenerating into `CompressedMapStatus` for large shuffle's.
    Btw, can we explicitly use math.max() ? instead of `v1 max v2` ? I thought that was the idiom used in spark ?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116926361
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -278,4 +278,39 @@ package object config {
             "spark.io.compression.codec.")
           .booleanConf
           .createWithDefault(false)
    +
    +  private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
    +    ConfigBuilder("spark.shuffle.accurateBlkThreshold")
    +      .doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will" +
    --- End diff --
    
    nit: you missed one space at the end


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115229968
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala ---
    @@ -128,4 +130,22 @@ class MapStatusSuite extends SparkFunSuite {
         assert(size1 === size2)
         assert(!success)
       }
    +
    +  test("Blocks which are bigger than 2 * average size should not be underestimated.") {
    +    val sizes = Array.concat(Array.fill[Long](1000)(1L), (1000L to 2000L).toArray)
    +    val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes)
    +    val arrayStream = new ByteArrayOutputStream(102400)
    +    val objectOutputStream = new ObjectOutputStream(arrayStream)
    +    assert(status1.isInstanceOf[HighlyCompressedMapStatus])
    +    objectOutputStream.writeObject(status1)
    +    objectOutputStream.flush()
    +    val array = arrayStream.toByteArray
    +    val objectInput = new ObjectInputStream(new ByteArrayInputStream(array))
    +    val status2 = objectInput.readObject().asInstanceOf[HighlyCompressedMapStatus]
    +    val avg = sizes.sum / 2001
    +    ((2 * avg + 1) to 2000).foreach {
    --- End diff --
    
    Yes, I should 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76686/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76485 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76485/testReport)** for PR 16989 at commit [`e09a123`](https://github.com/apache/spark/commit/e09a123bab51cdadcff87b4471d1d7672587b6fd).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115247947
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -154,15 +164,25 @@ final class ShuffleBlockFetcherIterator(
         while (iter.hasNext) {
           val result = iter.next()
           result match {
    -        case SuccessFetchResult(_, address, _, buf, _) =>
    +        case SuccessFetchResult(_, address, size, buf, _) =>
               if (address != blockManager.blockManagerId) {
                 shuffleMetrics.incRemoteBytesRead(buf.size)
                 shuffleMetrics.incRemoteBlocksFetched(1)
               }
               buf.release()
    +          freeMemory(size)
             case _ =>
           }
         }
    +    shuffleFiles.foreach {
    --- End diff --
    
    nit:
    ```
    shuffleFiles.foreach { f =>
      ...
    }
    ```


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r118272605
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -163,6 +170,11 @@ final class ShuffleBlockFetcherIterator(
             case _ =>
           }
         }
    +    shuffleFilesSet.foreach { file =>
    +      if (!file.delete()) {
    +        logInfo("Failed to cleanup " + file.getAbsolutePath());
    --- End diff --
    
    Yes, I should 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 pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115218041
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -163,6 +173,8 @@ final class ShuffleBlockFetcherIterator(
             case _ =>
           }
         }
    +    freeMemory(getUsed)
    --- End diff --
    
    to be more accurate, shall we free the memory in the while loop above?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76686 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76686/testReport)** for PR 16989 at commit [`52d08df`](https://github.com/apache/spark/commit/52d08dfd917bf9f4e7a97e8910af4eccdad07f45).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class FileSegmentManagedBuffer extends ManagedBuffer `


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75435 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75435/testReport)** for PR 16989 at commit [`03213aa`](https://github.com/apache/spark/commit/03213aa3e877b989c34a582d5aad36e1e21168e1).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77285 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77285/testReport)** for PR 16989 at commit [`222680c`](https://github.com/apache/spark/commit/222680c9d311f2d3fe7265fbf6e834e73cf4c05d).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    @cloud-fan 
    Thank you very much for reviewing this thus far :)
    >How about we always fetch to disk if the block size is over maxBytesInFlight?
    I super agree with this. It's to use `maxBytesInFlight` as a cap for controlling the memory. 
    
    I changed it to be below:
    ```
    // Shuffle remote blocks to disk when local memory shortage or the request is too large.
    if (acquired < req.size || bytesInFlight > maxBytesInFlight) {
      ...
      
    }
    ```


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76720 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76720/testReport)** for PR 16989 at commit [`af4ddf0`](https://github.com/apache/spark/commit/af4ddf05e820d0b33a408df71ebdc3a6aba93bff).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115826803
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +193,54 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large or local memory shortage.
    +    val fetchToDisk = if (bytesInFlight > maxBytesInFlight) {
    --- End diff --
    
    `maxBytesInFlight` is supposed to control how much data we want in flight (on network) - and not how much data should be used for shuffle in memory. 
    Prior to this change, there was a direct correlation due to the implementation details (you can only shuffle what you have read).
    
    Though related, they might need to be independently controlled - or are we tying them together for simplicity for now ? Any thoughts @cloud-fan ?
    
    For example: default value of maxBytesInFlight is 48mb iirc - which is can be really small for executors running on 10s or 100s of GB RAM !


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115247559
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala ---
    @@ -88,13 +89,15 @@ private[spark] class NettyBlockTransferService(
           port: Int,
           execId: String,
           blockIds: Array[String],
    -      listener: BlockFetchingListener): Unit = {
    +      listener: BlockFetchingListener,
    +      shuffleFilesOpt: Option[Array[File]]): Unit = {
    --- End diff --
    
    nit: shall we just use `Array[File]` and fetch data to memory if it's empty?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    +1 on @JoshRosen 's suggestion, we can integrate it with memory manager later.
    
    cc @JoshRosen shall we put this patch to branch 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 pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117174623
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -193,8 +217,19 @@ private[spark] object HighlyCompressedMapStatus {
         } else {
           0
         }
    +    val threshold = SparkEnv.get.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD)
    +    val hugeBlockSizesArray = ArrayBuffer[Tuple2[Int, Byte]]()
    --- End diff --
    
    If you're only going to be using `avgSize` for the non-huge blocks then I think you'd want to remove the huge blocks from the numerator in that calculation so that you more accurately size the smaller blocks, that way you're left with `averageOfNonEmptyNonHugeBlocks, preciseSizesOfHugeBlocks`.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    @cloud-fan
    Yes, thanks a lot for merging #18031 
    I will update soon !


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76486/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115410381
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -128,41 +130,52 @@ private[spark] class CompressedMapStatus(
      * @param numNonEmptyBlocks the number of non-empty blocks
      * @param emptyBlocks a bitmap tracking which blocks are empty
      * @param avgSize average size of the non-empty blocks
    + * @param hugeBlockSizesArray sizes of huge blocks by their reduceId.
      */
     private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizesArray: Array[Tuple2[Int, Byte]])
       extends MapStatus with Externalizable {
     
    +  @transient var hugeBlockSizes: Map[Int, Byte] =
    +    if (hugeBlockSizesArray == null) null else hugeBlockSizesArray.toMap
    +
       // loc could be null when the default constructor is called during deserialization
       require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
         "Average size can only be zero for map stages that produced no output")
     
    -  protected def this() = this(null, -1, null, -1)  // For deserialization only
    +  protected def this() = this(null, -1, null, -1, null)  // For deserialization only
     
       override def location: BlockManagerId = loc
     
       override def getSizeForBlock(reduceId: Int): Long = {
         if (emptyBlocks.contains(reduceId)) {
           0
         } else {
    -      avgSize
    +      hugeBlockSizes.get(reduceId) match {
    +        case Some(size) => MapStatus.decompressSize(size)
    +        case None => avgSize
    +      }
         }
       }
     
       override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
         loc.writeExternal(out)
         emptyBlocks.writeExternal(out)
         out.writeLong(avgSize)
    +    out.writeObject(hugeBlockSizesArray)
       }
     
       override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
         loc = BlockManagerId(in)
         emptyBlocks = new RoaringBitmap()
         emptyBlocks.readExternal(in)
         avgSize = in.readLong()
    +    hugeBlockSizesArray = in.readObject().asInstanceOf[Array[Tuple2[Int, Byte]]]
    --- End diff --
    
    This can be null, and so need to be handled appropriately below.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115230064
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -163,6 +173,8 @@ final class ShuffleBlockFetcherIterator(
             case _ =>
           }
         }
    +    freeMemory(getUsed)
    +    shuffleFiles.foreach(_.delete())
    --- End diff --
    
    Sorry, I will fix.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115219343
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala ---
    @@ -128,4 +130,22 @@ class MapStatusSuite extends SparkFunSuite {
         assert(size1 === size2)
         assert(!success)
       }
    +
    +  test("Blocks which are bigger than 2 * average size should not be underestimated.") {
    +    val sizes = Array.concat(Array.fill[Long](1000)(1L), (1000L to 2000L).toArray)
    +    val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes)
    +    val arrayStream = new ByteArrayOutputStream(102400)
    +    val objectOutputStream = new ObjectOutputStream(arrayStream)
    +    assert(status1.isInstanceOf[HighlyCompressedMapStatus])
    +    objectOutputStream.writeObject(status1)
    +    objectOutputStream.flush()
    +    val array = arrayStream.toByteArray
    +    val objectInput = new ObjectInputStream(new ByteArrayInputStream(array))
    +    val status2 = objectInput.readObject().asInstanceOf[HighlyCompressedMapStatus]
    +    val avg = sizes.sum / 2001
    +    ((2 * avg + 1) to 2000).foreach {
    --- End diff --
    
    add some comments to explain that the index of the `sizes` array is the value of that element.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115248267
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +195,47 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +    val acquired = acquireMemory(req.size)
    +    // Shuffle remote blocks to disk when local memory shortage or the request is too large.
    +    if (acquired < req.size || bytesInFlight > maxBytesInFlight) {
    --- End diff --
    
    if `bytesInFlight > maxBytesInFlight`, we don't need to bother the memory manager at all.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76486 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76486/testReport)** for PR 16989 at commit [`4b4e42e`](https://github.com/apache/spark/commit/4b4e42eb0cc340548fe4af0177736610aa59ec9f).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    @JoshRosen 
    Thanks a lot for taking time looking into this pr. I'm reading your comments carefully. 
    Yes, I think it's good to integrate with memory manager later. 
    I will break this pr into two smaller ones: one deals only with MapStatus compression accuracy improvements and another which forces blocks to disk over a certain fixed threshold. 
    Thanks again for your comments, very helpful :-) 👍 


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117172780
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +197,54 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +
    +    // Shuffle remote blocks to disk when the request is too large or local memory shortage.
    +    val fetchToDisk = if (req.size > maxReqSizeShuffleToMem) {
    +      true
    +    } else {
    +      val acquired = acquireMemory(req.size)
    +      if (acquired < req.size) {
    +        freeMemory(acquired)
    +        true
    +      } else {
    +        false
    +      }
    +    }
    +
    +    if (fetchToDisk) {
    +      shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    +        blockFetchingListener,
    +        blockIds.map(bId => blockManager.diskBlockManager.getFile(s"remote-$bId")).toArray)
    --- End diff --
    
    There's technically nothing which prohibits two tasks from fetching the same shuffle block. This doesn't happen in practice today, but there's nothing in Spark's shuffle interfaces which preclude an all-map-outputs-to-all-reduce-tasks broadcast. Given this, I'd prefer to be defensive and assign a unique temporary file name which incorporates the blockId but also includes something unique (such as the task attempt number).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116911752
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +429,146 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size of the request is above the" +
    +    " threshold(maxReqSizeShuffleToMem.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +          }
    +        }
    +      })
    +
    +    val taskMemoryManager = createMockTaskMemoryManager()
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator1 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress1,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles === null)
    +
    +    val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq))
    +    // Set maxReqSizeShuffleToMem to be 200.
    +    val iterator2 = new ShuffleBlockFetcherIterator(
    +      tc,
    +      transfer,
    +      blockManager,
    +      blocksByAddress2,
    +      (_, in) => in,
    +      Int.MaxValue,
    +      Int.MaxValue,
    +      200,
    +      true,
    +      taskMemoryManager)
    +    assert(shuffleFiles != null)
    +  }
    +
    +  test("Blocks should be shuffled to disk when size is above memory threshold," +
    +    " otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFiles: Array[File] = null
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFiles = invocation.getArguments()(5).asInstanceOf[Array[File]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        // 500 bytes at most can be offered from TaskMemoryManager.
    +        override def answer(invocationOnMock: InvocationOnMock): Long = {
    +          val required = invocationOnMock.getArguments()(0).asInstanceOf[Long]
    +          if (required <= 500) {
    +            return required
    +          } else {
    +            return 500
    --- End diff --
    
    sorry I didn't make it clear... I was wondering why we can pass test. We have 3 blocks with size 200, and all of them should be able to allocate memory, isn't it?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117169752
  
    --- Diff: common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java ---
    @@ -95,6 +97,25 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) {
       }
     
       @Override
    +  public ManagedBuffer openStream(String streamChunkId) {
    +    Tuple2<Long, Integer> streamIdAndChunkId = parseStreamChunkId(streamChunkId);
    +    return getChunk(streamIdAndChunkId._1, streamIdAndChunkId._2);
    +  }
    +
    +  public static String genStreamChunkId(long streamId, int chunkId) {
    +    return String.format("%d_%d", streamId, chunkId);
    +  }
    +
    +  public static Tuple2<Long, Integer> parseStreamChunkId(String streamChunkId) {
    --- End diff --
    
    This method is called in a single place. Because it needs to return two values I see that you had to import the Scala tuple class in this Java code. You could avoid the need to do this by simply inlining this at its sole call site, which I think would make the code simpler to read.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    I think that the current use of `MemoryMode.OFF_HEAP` allocation will cause problems in out-of-the-box deployments using the default configurations. In Spark's current memory manager implementation the total amount of Spark-managed off-heap memory that we will use is controlled by `spark.memory.offHeap.size` and the default value is 0. In this PR, the comment on `spark.reducer.maxReqSizeShuffleToMem` says that it should be smaller than `spark.memory.offHeap.size` and yet the default is 200 megabytes so the default configuration is invalid.
    
    Because `preferDirectBufs()` is `true` by default it looks like the code here will always try to reserve memory using `MemoryMode.OFF_HEAP` and these reservations will always fail in the default configuration because the off-heap size will be zero, so I think the net effect of this patch will be to always spill to disk.
    
    One way to address this problem is to configure the default value of `spark.memory.offHeap.size` to match the JVM's internal limit on the amount of direct buffers that it can allocate minus some percentage or fixed overhead. Basically the problem is that Spark's off-heap memory manager was originally designed to only manage off-heap memory explicitly allocated by Spark itself when creating its own buffers / pages or caching blocks, not to account for off-heap memory used by lower-level code or third-party libraries. I'll see if I can think of a clean way to fix this, which I think will need to be done before the defaults used here can work as intended.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r114948065
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -48,6 +58,8 @@
       private final String[] blockIds;
       private final BlockFetchingListener listener;
       private final ChunkReceivedCallback chunkCallback;
    +  private TransportConf transportConf = null;
    +  private File[] shuffleFiles = null;
    --- End diff --
    
    please follow the existing style, i.e. not assign values to these member variables.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76942 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76942/testReport)** for PR 16989 at commit [`958c220`](https://github.com/apache/spark/commit/958c2204c10c63d8699e4c16b2af6216bba00048).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    @cloud-fan
    In current change, the shuffle files are deleted twice:
    1). After the `ManagedBuffer.release`
    2). In the `cleanup()`, the `cleanup()` is already registered as a task completion callback.
    
    You mean that it's better to remove 1) ?
    In my understanding, there's no need to create another task completion callback. We just delete the files in `cleanup()`


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75935 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75935/testReport)** for PR 16989 at commit [`135c668`](https://github.com/apache/spark/commit/135c6687bfb22940a735305b8bcd8dfd489ce0bb).
     * 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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

    https://github.com/apache/spark/pull/16989#discussion_r114503557
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +147,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    +    }
    +
    +    @Override
    +    public void onComplete(String streamId) throws IOException {
    +      channel.close();
    +      ManagedBuffer buffer = new FileSegmentManagedBuffer(
    +        transportConf, targetFile, 0, targetFile.length());
    +      listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer);
    +    }
    +
    +    @Override
    +    public void onFailure(String streamId, Throwable cause) throws IOException {
    --- End diff --
    
    Yes, that will be good !


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r114953735
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala ---
    @@ -128,4 +130,23 @@ class MapStatusSuite extends SparkFunSuite {
         assert(size1 === size2)
         assert(!success)
       }
    +
    +  test("Blocks which are bigger than 2 * average size should not be underestimated.") {
    +    val sizes = Array.concat(Array.fill[Long](1000)(1L), (1000L to 2000L).toArray)
    +    val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes)
    +    val arrayStream = new ByteArrayOutputStream(102400)
    +    val objectOutputStream = new ObjectOutputStream(arrayStream)
    +    assert(status1.isInstanceOf[HighlyCompressedMapStatus])
    +    status1.asInstanceOf[HighlyCompressedMapStatus].writeExternal(objectOutputStream)
    +    objectOutputStream.flush()
    +    val array = arrayStream.toByteArray
    +    val objectInput = new ObjectInputStream(new ByteArrayInputStream(array))
    +    val status2 = new HighlyCompressedMapStatus()
    +    status2.readExternal(objectInput)
    --- End diff --
    
    nit: `val status2 = objectInput.readObject().asInstanceOf[HighlyCompressedMapStatus]`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    As @mridulm mentioned, in `HighlyCompressedMapStatus` it can be configured in two respects: 
    >1. minimum size before we consider something a large block.
    >2. The fraction '2' should also be configurable.
    
     I spent quite a while for thinking this and didn't come up with good names for these two configurations. @mridulm @cloud-fan Could you please give some advice ? How about `spark.shuffle.accurate.block.bound`, `spark.shuffle.accurate.block.multiples` - Actually, I think they are not good :-(  
    Should I put it in this pr or make another one ?


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75858/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115407998
  
    --- Diff: common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java ---
    @@ -95,6 +95,14 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) {
       }
     
       @Override
    +  public ManagedBuffer openStream(String streamChunkId) {
    +    String[] array = streamChunkId.split("_");
    --- End diff --
    
    Instead of spread the parsing logic, it is better to externalize this into a pair of methods - one to create streamChunkId given streamId and chunkIndex and another to retrieve it.
    If we have to change delimiter or add other logic, it will be more easier to manage the change.


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76622 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76622/testReport)** for PR 16989 at commit [`c58dcf4`](https://github.com/apache/spark/commit/c58dcf448723ea51d38bc07bf83c079a293c8d88).
     * 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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #75436 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75436/testReport)** for PR 16989 at commit [`65e7c42`](https://github.com/apache/spark/commit/65e7c42c00369eec6f1138848c3af0b0788f7937).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76721/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r165214410
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    --- End diff --
    
    right, I realize there isn't a simple one-line change here to switch to using spliceTo, I was wondering what the behavior is. 
    
    I actually thought zero-copy and offheap were orthogonal -- anytime netty gives you direct access to bytes, it has to be copied to user space, right?  


---

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


[GitHub] spark pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118186420
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -163,6 +170,11 @@ final class ShuffleBlockFetcherIterator(
             case _ =>
           }
         }
    +    shuffleFilesSet.foreach { file =>
    +      if (!file.delete()) {
    +        logInfo("Failed to cleanup " + file.getAbsolutePath());
    --- End diff --
    
    nit: `Failed to cleanup shuffle fetch temp file xxx`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76720/
    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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r116906367
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -278,4 +278,10 @@ package object config {
             "spark.io.compression.codec.")
           .booleanConf
           .createWithDefault(false)
    +  private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
    +    ConfigBuilder("spark.shuffle.accurateBlkThreshold")
    +      .doc("This only works in HighlyCompressedMapStatus. Sizes of blocks above this threshold" +
    +        " will be recorded accurately. This is to help control memory cost when shuffle blocks.")
    --- End diff --
    
    also update https://github.com/apache/spark/pull/16989/files#diff-76e731333fb756df3bff5ddb3b731c46R627


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

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


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r114954940
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +424,74 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size is above the threshold, otherwise to memory.") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    +    doReturn(diskBlockManager).when(blockManager).diskBlockManager
    +
    +    val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
    +    val remoteBlocks = Map[BlockId, ManagedBuffer](
    +      ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(),
    +      ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer())
    +    val transfer = mock(classOf[BlockTransferService])
    +    var shuffleFilesOpt: Option[Array[File]] = None
    +    when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
    +      .thenAnswer(new Answer[Unit] {
    +        override def answer(invocation: InvocationOnMock): Unit = {
    +          val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
    +          shuffleFilesOpt = invocation.getArguments()(5).asInstanceOf[Option[Array[File]]]
    +          Future {
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 1, 0).toString, remoteBlocks(ShuffleBlockId(0, 1, 0)))
    +            listener.onBlockFetchSuccess(
    +              ShuffleBlockId(0, 2, 0).toString, remoteBlocks(ShuffleBlockId(0, 2, 0)))
    +          }
    +        }
    +      })
    +    val taskMemoryManager = mock(classOf[TaskMemoryManager])
    +    when(taskMemoryManager.acquireExecutionMemory(any(), any()))
    +      .thenAnswer(new Answer[Long] {
    +        override def answer(invocationOnMock: InvocationOnMock): Long = 500L
    +      })
    +
    +    val tc = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null)
    +
    +    val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
    +      (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100.asInstanceOf[Long])).toSeq))
    --- End diff --
    
    `100.asInstanceOf[Long]` -> `100L`


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r117170538
  
    --- Diff: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java ---
    @@ -126,4 +150,50 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) {
           }
         }
       }
    +
    +  private class DownloadCallback implements StreamCallback {
    +
    +    private WritableByteChannel channel = null;
    +    private File targetFile = null;
    +    private int chunkIndex;
    +
    +    public DownloadCallback(File targetFile, int chunkIndex) throws IOException {
    +      this.targetFile = targetFile;
    +      this.channel = Channels.newChannel(new FileOutputStream(targetFile));
    +      this.chunkIndex = chunkIndex;
    +    }
    +
    +    @Override
    +    public void onData(String streamId, ByteBuffer buf) throws IOException {
    +      channel.write(buf);
    +    }
    +
    +    @Override
    +    public void onComplete(String streamId) throws IOException {
    +      channel.close();
    +      ManagedBuffer buffer = new FileSegmentManagedBuffer(
    +        transportConf, targetFile, 0, targetFile.length()) {
    +        @Override
    +        public ManagedBuffer release() {
    +          ManagedBuffer ret = super.release();
    +          if (!targetFile.delete()) {
    +            logger.info("Failed to cleanup " + targetFile.getAbsolutePath());
    +          }
    +          return ret;
    +        }
    +      };
    +      listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer);
    +    }
    +
    +    @Override
    +    public void onFailure(String streamId, Throwable cause) throws IOException {
    +      channel.close();
    +      // On receipt of a failure, fail every block from chunkIndex onwards.
    +      String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length);
    +      failRemainingBlocks(remainingBlockIds, cause);
    +      if (!targetFile.delete()) {
    +        logger.info("Failed to cleanup " + targetFile.getAbsolutePath());
    --- End diff --
    
    `warn` instead of `info`, perhaps?


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r118184231
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -287,4 +287,10 @@ package object config {
           .bytesConf(ByteUnit.BYTE)
           .createWithDefault(100 * 1024 * 1024)
     
    +  private[spark] val REDUCER_MAX_REQ_SIZE_SHUFFLE_TO_MEM =
    +    ConfigBuilder("spark.reducer.maxReqSizeShuffleToMem")
    +      .doc("The blocks of a shuffle request will be fetched to disk when size of the request is " +
    +        "above this threshold. This is to avoid a giant request takes too much memory.")
    +      .longConf
    +      .createWithDefault(200 * 1024 * 1024)
    --- End diff --
    
    shall we use `bytesConf(ByteUnit.MiB)` 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 pull request #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r118424375
  
    --- Diff: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala ---
    @@ -401,4 +411,61 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
         assert(id3 === ShuffleBlockId(0, 2, 0))
       }
     
    +  test("Blocks should be shuffled to disk when size of the request is above the" +
    +    " threshold(maxReqSizeShuffleToMem).") {
    +    val blockManager = mock(classOf[BlockManager])
    +    val localBmId = BlockManagerId("test-client", "test-client", 1)
    +    doReturn(localBmId).when(blockManager).blockManagerId
    +
    +    val diskBlockManager = mock(classOf[DiskBlockManager])
    +    doReturn(new File("shuffle-read-file")).when(diskBlockManager).getFile(any(classOf[String]))
    --- End diff --
    
    Yes, sorry for this stupid mistake ....


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76808/testReport)** for PR 16989 at commit [`778c59b`](https://github.com/apache/spark/commit/778c59bcc6dbf98d410a6fe3718594a6ecafbfbd).


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when ...

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

    https://github.com/apache/spark/pull/16989#discussion_r111734780
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala ---
    @@ -133,36 +135,53 @@ private[spark] class HighlyCompressedMapStatus private (
         private[this] var loc: BlockManagerId,
         private[this] var numNonEmptyBlocks: Int,
         private[this] var emptyBlocks: RoaringBitmap,
    -    private[this] var avgSize: Long)
    +    private[this] var avgSize: Long,
    +    private[this] var hugeBlockSizes: HashMap[Int, Byte])
       extends MapStatus with Externalizable {
     
       // loc could be null when the default constructor is called during deserialization
       require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
         "Average size can only be zero for map stages that produced no output")
     
    -  protected def this() = this(null, -1, null, -1)  // For deserialization only
    +  def this() = this(null, -1, null, -1, null)  // For deserialization only
    --- End diff --
    
    Remove the `protected` and make this visible for test.


---
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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #77245 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77245/testReport)** for PR 16989 at commit [`6c63a8b`](https://github.com/apache/spark/commit/6c63a8b87800b6c8916b906a35801ddfa4335fd6).
     * 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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    LGTM, only one comment: https://github.com/apache/spark/pull/16989#discussion_r118151720


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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/16989#discussion_r115219013
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala ---
    @@ -175,33 +187,45 @@ final class ShuffleBlockFetcherIterator(
         val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
         val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
         val blockIds = req.blocks.map(_._1.toString)
    -
         val address = req.address
    -    shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    -      new BlockFetchingListener {
    -        override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    -          // Only add the buffer to results queue if the iterator is not zombie,
    -          // i.e. cleanup() has not been called yet.
    -          ShuffleBlockFetcherIterator.this.synchronized {
    -            if (!isZombie) {
    -              // Increment the ref count because we need to pass this to a different thread.
    -              // This needs to be released after use.
    -              buf.retain()
    -              remainingBlocks -= blockId
    -              results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    -                remainingBlocks.isEmpty))
    -              logDebug("remainingBlocks: " + remainingBlocks)
    -            }
    +
    +    val blockFetchingListener = new BlockFetchingListener {
    +      override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
    +        // Only add the buffer to results queue if the iterator is not zombie,
    +        // i.e. cleanup() has not been called yet.
    +        ShuffleBlockFetcherIterator.this.synchronized {
    +          if (!isZombie) {
    +            // Increment the ref count because we need to pass this to a different thread.
    +            // This needs to be released after use.
    +            buf.retain()
    +            remainingBlocks -= blockId
    +            results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
    +              remainingBlocks.isEmpty))
    +            logDebug("remainingBlocks: " + remainingBlocks)
               }
    -          logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
             }
    +        logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
    +      }
     
    -        override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    -          logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    -          results.put(new FailureFetchResult(BlockId(blockId), address, e))
    -        }
    +      override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
    +        logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
    +        results.put(new FailureFetchResult(BlockId(blockId), address, e))
           }
    -    )
    +    }
    +    val acquired = acquireMemory(req.size)
    +    if (acquired < req.size) {
    +      freeMemory(acquired)
    +      shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
    +        blockFetchingListener,
    +        Some(blockIds.map(bId => {
    --- End diff --
    
    nit:
    ```
    Some(blockIds.map { bId => 
      ...
    })
    ```


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuffle-read...

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

    https://github.com/apache/spark/pull/16989
  
    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 #16989: [WIP][SPARK-19659] Fetch big blocks to disk when shuffle...

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

    https://github.com/apache/spark/pull/16989
  
    **[Test build #76415 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76415/testReport)** for PR 16989 at commit [`4045d6e`](https://github.com/apache/spark/commit/4045d6ea25bccdd347aeed7c3b042086822d587c).


---
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 #16989: [SPARK-19659] Fetch big blocks to disk when shuff...

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

    https://github.com/apache/spark/pull/16989#discussion_r115018506
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala ---
    @@ -128,4 +130,23 @@ class MapStatusSuite extends SparkFunSuite {
         assert(size1 === size2)
         assert(!success)
       }
    +
    +  test("Blocks which are bigger than 2 * average size should not be underestimated.") {
    +    val sizes = Array.concat(Array.fill[Long](1000)(1L), (1000L to 2000L).toArray)
    +    val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes)
    +    val arrayStream = new ByteArrayOutputStream(102400)
    +    val objectOutputStream = new ObjectOutputStream(arrayStream)
    +    assert(status1.isInstanceOf[HighlyCompressedMapStatus])
    +    status1.asInstanceOf[HighlyCompressedMapStatus].writeExternal(objectOutputStream)
    --- End diff --
    
    Yes, I should refine this.


---
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