You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by davies <gi...@git.apache.org> on 2015/10/23 08:48:56 UTC

[GitHub] spark pull request: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

GitHub user davies opened a pull request:

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

    [SPARK-10342] [SQL] [WIP] Cooperative memory management

    WIP

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

    $ git pull https://github.com/davies/spark force_spill

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

    https://github.com/apache/spark/pull/9241.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 #9241
    
----
commit 758b4b7fda78db1fffe3ac9f10f6409b453fbc25
Author: Davies Liu <da...@databricks.com>
Date:   2015-10-22T23:26:37Z

    force spilling

commit 35624760abe0a1565590d94e9b6ff6746dfaed26
Author: Davies Liu <da...@databricks.com>
Date:   2015-10-23T06:44:49Z

    Merge branch 'master' of github.com:apache/spark into force_spill
    
    Conflicts:
    	core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151570795
  
    **[Test build #44447 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44447/consoleFull)** for PR 9241 at commit [`d0ada7b`](https://github.com/apache/spark/commit/d0ada7b5c6f8be9a91624e4eb5a6ca2242edf16b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152299246
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44613/
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150701017
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152297200
  
    **[Test build #44629 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44629/consoleFull)** for PR 9241 at commit [`4491013`](https://github.com/apache/spark/commit/4491013c9b669d7b03fbe16c7d68f62f28175f7e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43452672
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords--;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    +        ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
    +
    +        long released = 0L;
    +        while (dataPages.size() > 0) {
    +          MemoryBlock block = dataPages.getLast();
    +          // The currentPage is used, cannot be released
    +          if (block == currentPage) {
    +            break;
    +          }
    +
    +          Object base = block.getBaseObject();
    +          long offset = block.getBaseOffset();
    +          int numRecords = Platform.getInt(base, offset);
    +          offset += 4;
    +          final UnsafeSorterSpillWriter writer =
    +            new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords);
    +          while (numRecords-- > 0) {
    --- End diff --
    
    I can never remember the ordering of these types of comparisons. Could you update the code to separate the assignment and comparison just so that it's absolutely unambiguous?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43448501
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala ---
    @@ -193,10 +193,6 @@ class UnsafeFixedWidthAggregationMapSuite
         // Convert the map into a sorter
         val sorter = map.destructAndCreateExternalSorter()
     
    -    withClue(s"destructAndCreateExternalSorter should release memory used by the map") {
    --- End diff --
    
    Yep, looks like we now account for the pointer array.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150519962
  
    **[Test build #44210 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44210/consoleFull)** for PR 9241 at commit [`3562476`](https://github.com/apache/spark/commit/35624760abe0a1565590d94e9b6ff6746dfaed26).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class BytesToBytesMapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n  * `abstract class MemoryConsumer `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#discussion_r43220734
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,100 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public class MemoryConsumer {
    --- End diff --
    
    Yes, we could make it abstract 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151992575
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152089158
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150753354
  
    **[Test build #44275 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44275/consoleFull)** for PR 9241 at commit [`86e47ca`](https://github.com/apache/spark/commit/86e47ca4a459fce086fb9a499189c19fb4044111).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43464779
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -227,62 +238,147 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class BytesToBytesMapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters =
    +      new LinkedList<UnsafeSorterSpillWriter>();
    +    private UnsafeSorterSpillReader reader = null;
    +
    +    private BytesToBytesMapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    -      }
    +      destructiveIterator = this;
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    -        this.bmap.shuffleMemoryManager.release(currentPage.size());
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          taskMemoryManager.freePage(currentPage);
    +          shuffleMemoryManager.release(currentPage.size());
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          try {
    +            reader = spillWriters.removeFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords --;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    --- End diff --
    
    Any final verdict on the `ShuffleWriteMetrics`? Want to just file a followup JIRA to audit this before 1.6?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43442805
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -261,17 +397,14 @@ public long getOffsetInPage(long pagePlusOffsetAddress) {
        * value can be used to detect memory leaks.
        */
       public long cleanUpAllAllocatedMemory() {
    -    long freedBytes = 0;
    -    for (MemoryBlock page : pageTable) {
    -      if (page != null) {
    -        freedBytes += page.size();
    -        freePage(page);
    +    synchronized (this) {
    +      Arrays.fill(pageTable, null);
    +      for (MemoryConsumer c: consumers.keySet()) {
    +        logger.warn("leak " + Utils.bytesToString(consumers.get(c)) + " memory from " + c);
    --- End diff --
    
    Just to clarify: is the memory leaked by these consumers also accounted for in the value returned by `releaseAllExecutionMemoryForTask`? Just want to make sure that this method's return value doesn't under- or over-count leaked memory.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43449396
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    --- End diff --
    
    Trying to pack into the wasted space at the end of pages, you mean?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150701034
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150738853
  
    **[Test build #44265 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44265/consoleFull)** for PR 9241 at commit [`5c198cf`](https://github.com/apache/spark/commit/5c198cf329748a0af9f108d182615616334a6c6a).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class BytesToBytesMapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n  * `abstract class MemoryConsumer `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151994205
  
    **[Test build #44542 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44542/consoleFull)** for PR 9241 at commit [`827d4f0`](https://github.com/apache/spark/commit/827d4f0a6d42e4f2af37eedacf3098fceb896c32).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43444037
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -261,17 +397,14 @@ public long getOffsetInPage(long pagePlusOffsetAddress) {
        * value can be used to detect memory leaks.
        */
       public long cleanUpAllAllocatedMemory() {
    -    long freedBytes = 0;
    -    for (MemoryBlock page : pageTable) {
    -      if (page != null) {
    -        freedBytes += page.size();
    -        freePage(page);
    +    synchronized (this) {
    +      Arrays.fill(pageTable, null);
    --- 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152034328
  
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151736450
  
    **[Test build #44504 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44504/consoleFull)** for PR 9241 at commit [`8470fc9`](https://github.com/apache/spark/commit/8470fc9ddd37c525cf648c00e019d968410fa66f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151701001
  
    **[Test build #44481 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44481/consoleFull)** for PR 9241 at commit [`97935e5`](https://github.com/apache/spark/commit/97935e560453dcfa0e4cf98ab0c54da39d6b612a).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151673382
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152333073
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43457661
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +105,103 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashSet<MemoryConsumer> consumers;
    --- End diff --
    
    Add a comment to explain that this field is guarded by synchronizing on `this` (or use `@GuardedBy("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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152286836
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152111686
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44569/
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151570051
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151688530
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43318337
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public abstract class MemoryConsumer {
    --- End diff --
    
    The length is about the same as TaskMemoryManager - so not too long.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152126836
  
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151701053
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44481/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152296883
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152400875
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43454081
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    --- End diff --
    
    Should also check that the read path also handles multiple spills.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953209
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIteratorSuite.scala ---
    @@ -1,54 +0,0 @@
    -/*
    - * Licensed to the Apache Software Foundation (ASF) under one or more
    - * contributor license agreements.  See the NOTICE file distributed with
    - * this work for additional information regarding copyright ownership.
    - * The ASF licenses this file to You under the Apache License, Version 2.0
    - * (the "License"); you may not use this file except in compliance with
    - * the License.  You may obtain a copy of the License at
    - *
    - *    http://www.apache.org/licenses/LICENSE-2.0
    - *
    - * Unless required by applicable law or agreed to in writing, software
    - * distributed under the License is distributed on an "AS IS" BASIS,
    - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    - * See the License for the specific language governing permissions and
    - * limitations under the License.
    - */
    -
    -package org.apache.spark.sql.execution.aggregate
    -
    -import org.apache.spark._
    -import org.apache.spark.sql.catalyst.expressions._
    -import org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection
    -import org.apache.spark.sql.execution.metric.SQLMetrics
    -import org.apache.spark.sql.test.SharedSQLContext
    -import org.apache.spark.unsafe.memory.TaskMemoryManager
    -
    -class TungstenAggregationIteratorSuite extends SparkFunSuite with SharedSQLContext {
    -
    -  test("memory acquired on construction") {
    --- End diff --
    
    This is just a consequence of no longer needing the reserved page now that we have force-spilling support, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152346305
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458691
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -353,15 +320,22 @@ private void growPointerArrayIfNecessary() throws IOException {
         assert(inMemSorter != null);
         if (!inMemSorter.hasSpaceForAnotherRecord()) {
           logger.debug("Attempting to expand sort pointer array");
    -      final long oldPointerArrayMemoryUsage = inMemSorter.getMemoryUsage();
    -      final long memoryToGrowPointerArray = oldPointerArrayMemoryUsage * 2;
    -      final long memoryAcquired = taskMemoryManager.acquireExecutionMemory(memoryToGrowPointerArray);
    -      if (memoryAcquired < memoryToGrowPointerArray) {
    -        taskMemoryManager.releaseExecutionMemory(memoryAcquired);
    -        spill();
    -      } else {
    -        inMemSorter.expandPointerArray();
    -        taskMemoryManager.releaseExecutionMemory(oldPointerArrayMemoryUsage);
    +      long used = inMemSorter.getMemoryUsage();
    +      long needed = inMemSorter.getMemoryToExpand();
    +      try {
    +        acquireMemory(used + needed);  // could trigger spilling
    +        if (inMemSorter.hasSpaceForAnotherRecord()) {
    +          releaseMemory(used + needed);
    +        } else {
    +          logger.debug("Expand sort pointer array");
    +          inMemSorter.expandPointerArray();
    +          releaseMemory(used);
    +        }
    +      } catch (OutOfMemoryError oom) {
    --- End diff --
    
    Couldn't this leave the JVM in an inconsistent state? Or is the idea that `OutOfMemoryError`s thrown in this thread will perhaps be recoverable? If we're really running low on memory, I suppose there's a good chance that other threads could hit OOM as well. I guess that those threads won't have their OOMs caught and will still be able to halt the JVM.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152353139
  
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150753360
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44275/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152338544
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43451271
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -353,15 +320,22 @@ private void growPointerArrayIfNecessary() throws IOException {
         assert(inMemSorter != null);
         if (!inMemSorter.hasSpaceForAnotherRecord()) {
           logger.debug("Attempting to expand sort pointer array");
    -      final long oldPointerArrayMemoryUsage = inMemSorter.getMemoryUsage();
    -      final long memoryToGrowPointerArray = oldPointerArrayMemoryUsage * 2;
    -      final long memoryAcquired = taskMemoryManager.acquireExecutionMemory(memoryToGrowPointerArray);
    -      if (memoryAcquired < memoryToGrowPointerArray) {
    -        taskMemoryManager.releaseExecutionMemory(memoryAcquired);
    -        spill();
    -      } else {
    -        inMemSorter.expandPointerArray();
    -        taskMemoryManager.releaseExecutionMemory(oldPointerArrayMemoryUsage);
    +      long used = inMemSorter.getMemoryUsage();
    +      long needed = inMemSorter.getMemoryToExpand();
    +      try {
    +        acquireMemory(used + needed);  // could trigger spilling
    +        if (inMemSorter.hasSpaceForAnotherRecord()) {
    +          releaseMemory(used + needed);
    +        } else {
    +          logger.debug("Expand sort pointer array");
    +          inMemSorter.expandPointerArray();
    +          releaseMemory(used);
    +        }
    +      } catch (OutOfMemoryError oom) {
    --- End diff --
    
    Actual it's better to also catch the OOM from JVM, in the case that we thought we have enough memory but actually it's not. I hit this problem during higher workload 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 pull request: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152112973
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152373513
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44644/
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151755200
  
    **[Test build #44491 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44491/consoleFull)** for PR 9241 at commit [`ce24f03`](https://github.com/apache/spark/commit/ce24f039f07bde78500191a9afec7a6298a6b559).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#discussion_r43187836
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,95 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      long old = 0L;
    +      if (consumers.containsKey(consumer)) {
    +        old = consumers.get(consumer);
    +      }
    +      consumers.put(consumer, got + old);
    +
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    -    memoryManager.releaseExecutionMemory(size, taskAttemptId);
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    +    if (size == 0) {
    +      return;
    +    }
    +    synchronized (this) {
    +      if (consumers.containsKey(consumer)) {
    +        long old = consumers.get(consumer);
    +        if (old > size) {
    +          consumers.put(consumer, old - size);
    +        } else {
    +          if (old < size) {
    +            if (Utils.isTesting()) {
    +              Platform.throwException(
    +                new SparkException("Release more memory " + size + "than acquired " + old + " for "
    +                  + consumer));
    +            } else {
    +              logger.warn("Release more memory " + size + " than acquired " + old + "for "
    --- End diff --
    
    Space before `for`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150491532
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152385812
  
    **[Test build #44647 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44647/consoleFull)** for PR 9241 at commit [`cda4b2a`](https://github.com/apache/spark/commit/cda4b2afa45acef529eb24afa7ec0c5370de237f).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151735339
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953580
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -279,13 +259,21 @@ void spill() throws IOException {
           spills.size() > 1 ? " times" : " time");
     
         writeSortedFile(false);
    -    final long inMemSorterMemoryUsage = inMemSorter.getMemoryUsage();
    -    inMemSorter = null;
    -    shuffleMemoryManager.release(inMemSorterMemoryUsage);
    +    numRecordsInsertedSinceLastSpill = 0;
         final long spillSize = freeMemory();
         taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
    +  }
     
    -    initializeForWriting();
    +  public long release(long numBytes) throws IOException {
    --- End diff --
    
    Add `@Override`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43442179
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // consumers could be modified by spill(), so we should have a copy here.
    +        MemoryConsumer[] cs = new MemoryConsumer[consumers.size()];
    +        consumers.keySet().toArray(cs);
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: cs) {
    +          if (c != null && c != consumer) {
    --- End diff --
    
    How do we wind up with null keys in the consumers map?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43442536
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // consumers could be modified by spill(), so we should have a copy here.
    +        MemoryConsumer[] cs = new MemoryConsumer[consumers.size()];
    +        consumers.keySet().toArray(cs);
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: cs) {
    +          if (c != null && c != consumer) {
    +            try {
    +              long released = c.spill(required - got, consumer);
    +              if (released > 0) {
    +                logger.info("Task {} released {} from {} for {}", taskAttemptId,
    +                  Utils.bytesToString(released), c, consumer);
    +                got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +                if (got >= required) {
    +                  break;
    +                }
    +              }
    +            } catch (IOException e) {
    +              logger.error("error while calling spill() on " + c, e);
    +              throw new OutOfMemoryError("error while calling spill() on " + c + " : "
    +                + e.getMessage());
    +            }
    +          }
    +        }
    +      }
    +
    +      // call spill() on itself
    +      if (got < required && consumer != null) {
    +        try {
    +          long released = consumer.spill(required - got, consumer);
    +          if (released > 0) {
    +            logger.info("Task {} released {} from itself ({})", taskAttemptId,
    +              Utils.bytesToString(released), consumer);
    +            got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +          }
    +        } catch (IOException e) {
    +          logger.error("error while calling spill() on " + consumer, e);
    +          throw new OutOfMemoryError("error while calling spill() on " + consumer + " : "
    +            + e.getMessage());
    +        }
    +      }
    +
    +      // Update the accounting, even consumer is null
    +      if (got > 0) {
    +        long old = 0L;
    +        if (consumers.containsKey(consumer)) {
    +          old = consumers.get(consumer);
    +        }
    +        consumers.put(consumer, got + old);
    +      }
    +
    +      logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer);
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    +    assert(size >= 0);
    +    if (size == 0) {
    +      return;
    +    }
    +    synchronized (this) {
    +      if (consumers.containsKey(consumer)) {
    +        long old = consumers.get(consumer);
    +        if (old > size) {
    +          consumers.put(consumer, old - size);
    +        } else {
    +          if (old < size) {
    +            String msg = "Release " + size + " bytes memory (more than acquired " + old + ") for "
    +              + consumer;
    +            logger.warn(msg);
    +            if (Utils.isTesting()) {
    +              Platform.throwException(new SparkException(msg));
    +            }
    +          }
    +          consumers.remove(consumer);
    +        }
    +      } else {
    +        String msg = "Release " + size + " bytes memory for non-existent " + consumer;
    +        logger.warn(msg);
    +        if (Utils.isTesting()) {
    +          Platform.throwException(new SparkException(msg));
    +        }
    +      }
    +    }
    +
    +    logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer);
         memoryManager.releaseExecutionMemory(size, taskAttemptId);
       }
     
    +  public void transferOwnership(long size, MemoryConsumer from, MemoryConsumer to) {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      if (consumers.containsKey(from)) {
    +        long old = consumers.get(from);
    +        if (old > size) {
    +          consumers.put(from, old - size);
    +        } else {
    +          consumers.remove(from);
    +        }
    +        if (consumers.containsKey(to)) {
    +          old = consumers.get(to);
    +        } else {
    +          old = 0L;
    +        }
    +        consumers.put(to, old + size);
    +      }
    --- End diff --
    
    Is the `else` branch here supposed to be an error case?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150703409
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152400814
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151683361
  
    **[Test build #44474 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44474/consoleFull)** for PR 9241 at commit [`7087f2f`](https://github.com/apache/spark/commit/7087f2f4ca9841716b9b4740c037b410607cf719).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#discussion_r43187499
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,95 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private HashMap<MemoryConsumer, Long> consumers;
    --- End diff --
    
    Could be final.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43311963
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,97 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got, consumer);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got, consumer);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      long old = 0L;
    +      if (consumers.containsKey(consumer)) {
    +        old = consumers.get(consumer);
    +      }
    +      consumers.put(consumer, got + old);
    +
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    -    memoryManager.releaseExecutionMemory(size, taskAttemptId);
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    +    assert(size >= 0);
    +    if (size == 0) {
    +      return;
    +    }
    +    synchronized (this) {
    +      if (consumers.containsKey(consumer)) {
    +        long old = consumers.get(consumer);
    +        if (old > size) {
    +          consumers.put(consumer, old - size);
    +        } else {
    +          if (old < size) {
    +            if (Utils.isTesting()) {
    +              Platform.throwException(
    +                new SparkException("Release more memory " + size + "than acquired " + old + " for "
    +                  + consumer));
    +            } else {
    +              logger.warn("Release more memory " + size + " than acquired " + old + " for "
    +                + consumer);
    +            }
    +          }
    +          consumers.remove(consumer);
    +        }
    +      } else {
    +        if (Utils.isTesting()) {
    +          Platform.throwException(
    --- End diff --
    
    Maybe add new method to Utils, accepting message String, which covers lines 191 to 196


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150701808
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152385983
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44647/
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151688512
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150727321
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43449674
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    --- End diff --
    
    Yes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43456699
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +105,103 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashSet<MemoryConsumer> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashSet<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers) {
    --- End diff --
    
    Does this approach still have the same concern about concurrent modification of `consumers` while iterating over 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43311814
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,97 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got, consumer);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got, consumer);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      long old = 0L;
    +      if (consumers.containsKey(consumer)) {
    +        old = consumers.get(consumer);
    +      }
    +      consumers.put(consumer, got + old);
    +
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    -    memoryManager.releaseExecutionMemory(size, taskAttemptId);
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    +    assert(size >= 0);
    +    if (size == 0) {
    +      return;
    +    }
    +    synchronized (this) {
    +      if (consumers.containsKey(consumer)) {
    +        long old = consumers.get(consumer);
    +        if (old > size) {
    +          consumers.put(consumer, old - size);
    +        } else {
    +          if (old < size) {
    +            if (Utils.isTesting()) {
    +              Platform.throwException(
    +                new SparkException("Release more memory " + size + "than acquired " + old + " for "
    +                  + consumer));
    +            } else {
    +              logger.warn("Release more memory " + size + " than acquired " + old + " for "
    +                + consumer);
    +            }
    +          }
    +          consumers.remove(consumer);
    +        }
    +      } else {
    +        if (Utils.isTesting()) {
    +          Platform.throwException(
    +            new SparkException("Release memory " + size + " for not existed " + consumer));
    --- End diff --
    
    not existed -> non-existent


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150753359
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43461487
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -531,113 +645,59 @@ public int getValueLength() {
          * @return true if the put() was successful and false if the put() failed because memory could
          *         not be acquired.
          */
    -    public boolean putNewKey(
    -        Object keyBaseObject,
    -        long keyBaseOffset,
    -        int keyLengthBytes,
    -        Object valueBaseObject,
    -        long valueBaseOffset,
    -        int valueLengthBytes) {
    +    public boolean putNewKey(Object keyBase, long keyOffset, int keyLength,
    +        Object valueBase, long valueOffset, int valueLength) {
           assert (!isDefined) : "Can only set value once for a key";
    -      assert (keyLengthBytes % 8 == 0);
    -      assert (valueLengthBytes % 8 == 0);
    +      assert (keyLength % 8 == 0);
    +      assert (valueLength % 8 == 0);
           assert(bitset != null);
           assert(longArray != null);
     
    -      if (numElements == MAX_CAPACITY) {
    -        throw new IllegalStateException("BytesToBytesMap has reached maximum capacity");
    +      if (numElements == MAX_CAPACITY || !canGrowArray) {
    +        return false;
           }
     
           // Here, we'll copy the data into our data pages. Because we only store a relative offset from
           // the key address instead of storing the absolute address of the value, the key and value
           // must be stored in the same memory page.
           // (8 byte key length) (key) (value)
    -      final long requiredSize = 8 + keyLengthBytes + valueLengthBytes;
    -
    -      // --- Figure out where to insert the new record ---------------------------------------------
    -
    -      final MemoryBlock dataPage;
    -      final Object dataPageBaseObject;
    -      final long dataPageInsertOffset;
    -      boolean useOverflowPage = requiredSize > pageSizeBytes - 8;
    -      if (useOverflowPage) {
    -        // The record is larger than the page size, so allocate a special overflow page just to hold
    -        // that record.
    -        final long overflowPageSize = requiredSize + 8;
    -        MemoryBlock overflowPage = taskMemoryManager.allocatePage(overflowPageSize);
    -        if (overflowPage == null) {
    -          logger.debug("Failed to acquire {} bytes of memory", overflowPageSize);
    -          return false;
    -        }
    -        dataPages.add(overflowPage);
    -        dataPage = overflowPage;
    -        dataPageBaseObject = overflowPage.getBaseObject();
    -        dataPageInsertOffset = overflowPage.getBaseOffset();
    -      } else if (currentDataPage == null || pageSizeBytes - 8 - pageCursor < requiredSize) {
    -        // The record can fit in a data page, but either we have not allocated any pages yet or
    -        // the current page does not have enough space.
    -        if (currentDataPage != null) {
    -          // There wasn't enough space in the current page, so write an end-of-page marker:
    -          final Object pageBaseObject = currentDataPage.getBaseObject();
    -          final long lengthOffsetInPage = currentDataPage.getBaseOffset() + pageCursor;
    -          Platform.putInt(pageBaseObject, lengthOffsetInPage, END_OF_PAGE_MARKER);
    -        }
    -        if (!acquireNewPage()) {
    +      final long recordLength = 8 + keyLength + valueLength;
    +      if (currentPage == null || currentPage.size() - pageCursor < recordLength) {
    +        if (!acquireNewPage(recordLength + 4L)) {
               return false;
             }
    -        dataPage = currentDataPage;
    -        dataPageBaseObject = currentDataPage.getBaseObject();
    -        dataPageInsertOffset = currentDataPage.getBaseOffset();
    -      } else {
    -        // There is enough space in the current data page.
    -        dataPage = currentDataPage;
    -        dataPageBaseObject = currentDataPage.getBaseObject();
    -        dataPageInsertOffset = currentDataPage.getBaseOffset() + pageCursor;
           }
     
           // --- Append the key and value data to the current data page --------------------------------
    -
    -      long insertCursor = dataPageInsertOffset;
    -
    -      // Compute all of our offsets up-front:
    -      final long recordOffset = insertCursor;
    -      insertCursor += 4;
    -      final long keyLengthOffset = insertCursor;
    -      insertCursor += 4;
    -      final long keyDataOffsetInPage = insertCursor;
    -      insertCursor += keyLengthBytes;
    -      final long valueDataOffsetInPage = insertCursor;
    -      insertCursor += valueLengthBytes; // word used to store the value size
    -
    -      Platform.putInt(dataPageBaseObject, recordOffset,
    -        keyLengthBytes + valueLengthBytes + 4);
    -      Platform.putInt(dataPageBaseObject, keyLengthOffset, keyLengthBytes);
    -      // Copy the key
    -      Platform.copyMemory(
    -        keyBaseObject, keyBaseOffset, dataPageBaseObject, keyDataOffsetInPage, keyLengthBytes);
    -      // Copy the value
    -      Platform.copyMemory(valueBaseObject, valueBaseOffset, dataPageBaseObject,
    -        valueDataOffsetInPage, valueLengthBytes);
    -
    -      // --- Update bookeeping data structures -----------------------------------------------------
    -
    -      if (useOverflowPage) {
    -        // Store the end-of-page marker at the end of the data page
    -        Platform.putInt(dataPageBaseObject, insertCursor, END_OF_PAGE_MARKER);
    -      } else {
    -        pageCursor += requiredSize;
    -      }
    -
    +      final Object base = currentPage.getBaseObject();
    +      long offset = currentPage.getBaseOffset() + pageCursor;
    +      final long recordOffset = offset;
    +      Platform.putInt(base, offset, keyLength + valueLength + 4);
    +      Platform.putInt(base, offset + 4, keyLength);
    +      offset += 8;
    +      Platform.copyMemory(keyBase, keyOffset, base, offset, keyLength);
    +      offset += keyLength;
    +      Platform.copyMemory(valueBase, valueOffset, base, offset, valueLength);
    +
    +      // --- Update bookkeeping data structures -----------------------------------------------------
    +      offset = currentPage.getBaseOffset();
    +      Platform.putInt(base, offset, Platform.getInt(base, offset) + 1);
    +      pageCursor += recordLength;
           numElements++;
           bitset.set(pos);
           final long storedKeyAddress = taskMemoryManager.encodePageNumberAndOffset(
    -        dataPage, recordOffset);
    +        currentPage, recordOffset);
           longArray.set(pos * 2, storedKeyAddress);
           longArray.set(pos * 2 + 1, keyHashcode);
           updateAddressesAndSizes(storedKeyAddress);
           isDefined = true;
    +
           if (numElements > growthThreshold && longArray.size() < MAX_CAPACITY) {
    -        growAndRehash();
    +        try {
    +          growAndRehash();
    +        } catch (OutOfMemoryError oom) {
    --- End diff --
    
    As a counter-argument to catching OutOfMemoryError: we have to treat OOM as an interrupt which can occur anywhere that we might be allocating memory in the methods that we call here, so an adversary could choose to have those OOMs occur in really bad places.
    
    Imagine that we're in growAndRehash, which calls `allocate()` to obtain a new bit set and array: https://github.com/davies/spark/blob/cda4b2afa45acef529eb24afa7ec0c5370de237f/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java#L865
    
    Imagine that we've mutated `longArray` and then throw an OOM when trying to mutate the `bitset`: https://github.com/davies/spark/blob/cda4b2afa45acef529eb24afa7ec0c5370de237f/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java#L742
    
    In this case, I think that the map will be in an inconsistent state.
    
    So in summary, I'm not necessarily opposed to the idea of catching OOM in a small handful of places, but I'm kind of wary about doing it due to subtle corner-cases like 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43464618
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +230,158 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    --- End diff --
    
    Why set this to -1?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152373451
  
    **[Test build #44644 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44644/consoleFull)** for PR 9241 at commit [`afc8c7c`](https://github.com/apache/spark/commit/afc8c7c9b0e92f9db6a5f72c14d8484a87311a51).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150684163
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44242/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152255537
  
    **[Test build #44613 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44613/consoleFull)** for PR 9241 at commit [`51278f8`](https://github.com/apache/spark/commit/51278f89d73d46f8cf22377c3bd0b7aa3ad5c02e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43318175
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,97 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got, consumer);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got, consumer);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    --- End diff --
    
    Then this call will be no-op, could be avoided.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43441866
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    --- End diff --
    
    Rather than maintaining a separate bookkeeping map, why not track the amount of memory granted to each `MemoryConsumer` inside of the consumer itself, then expose a method to let us retrieve that size from the consumer instance?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152338462
  
    **[Test build #44629 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44629/consoleFull)** for PR 9241 at commit [`4491013`](https://github.com/apache/spark/commit/4491013c9b669d7b03fbe16c7d68f62f28175f7e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43450454
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    +      currentPage = allocatePage(required);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
        * Write a record to the shuffle sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      int partitionId) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId)
    +    throws IOException {
     
    -    if (numRecordsInsertedSinceLastSpill > numElementsForSpillThreshold) {
    +    // for tests
    +    assert(inMemSorter != null);
    +    if (inMemSorter.numRecords() > numElementsForSpillThreshold) {
           spill();
         }
     
         growPointerArrayIfNecessary();
    -    // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    -      long overflowPageSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(totalSpaceRequired);
    -      // The record is larger than the page size, so allocate a special overflow page just to hold
    -      // that record.
    -      MemoryBlock overflowPage = taskMemoryManager.allocatePage(overflowPageSize);
    -      if (overflowPage == null) {
    -        spill();
    -        overflowPage = taskMemoryManager.allocatePage(overflowPageSize);
    -        if (overflowPage == null) {
    -          throw new IOException("Unable to acquire " + overflowPageSize + " bytes of memory");
    -        }
    -      }
    -      allocatedPages.add(overflowPage);
    -      dataPage = overflowPage;
    -      dataPagePosition = overflowPage.getBaseOffset();
    -    } else {
    -      // The record is small enough to fit in a regular data page, but the current page might not
    -      // have enough space to hold it (or no pages have been allocated yet).
    -      acquireNewPageIfNecessary(totalSpaceRequired);
    -      dataPage = currentPage;
    -      dataPagePosition = currentPagePosition;
    -      // Update bookkeeping information
    -      freeSpaceInCurrentPage -= totalSpaceRequired;
    -      currentPagePosition += totalSpaceRequired;
    -    }
    -    final Object dataPageBaseObject = dataPage.getBaseObject();
    -
    -    final long recordAddress =
    -      taskMemoryManager.encodePageNumberAndOffset(dataPage, dataPagePosition);
    -    Platform.putInt(dataPageBaseObject, dataPagePosition, lengthInBytes);
    -    dataPagePosition += 4;
    -    Platform.copyMemory(
    -      recordBaseObject, recordBaseOffset, dataPageBaseObject, dataPagePosition, lengthInBytes);
    -    assert(inMemSorter != null);
    +    final int required = length + 4;
    --- End diff --
    
    I'd add back the now-missing comment that says "Need 4 bytes to store the record length."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43460412
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java ---
    @@ -295,8 +290,22 @@ public void cleanupResources() {
       private void growPointerArrayIfNecessary() throws IOException {
         assert(inMemSorter != null);
         if (!inMemSorter.hasSpaceForAnotherRecord()) {
    -      // TODO: track the pointer array memory! (SPARK-10474)
    -      inMemSorter.expandPointerArray();
    +      long used = inMemSorter.getMemoryUsage();
    +      long needed = inMemSorter.getMemoryToExpand();
    +      try {
    +        acquireMemory(used + needed);  // could trigger spilling
    +        if (inMemSorter.hasSpaceForAnotherRecord()) {
    --- End diff --
    
    Discussed offline. The current logic is correct because acquireMemory() might have caused the in-memory sorter to spill.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152436900
  
    **[Test build #44656 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44656/consoleFull)** for PR 9241 at commit [`e943e74`](https://github.com/apache/spark/commit/e943e745317bd4d13934a14026be5a7915706269).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152254296
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152098140
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150684105
  
    **[Test build #44242 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44242/consoleFull)** for PR 9241 at commit [`0c77c94`](https://github.com/apache/spark/commit/0c77c94c9d50ad8935a5b3009a25e21966193987).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class BytesToBytesMapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n  * `abstract class MemoryConsumer `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43443271
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public abstract class MemoryConsumer {
    --- End diff --
    
    I'm neutral on the name change. At first I thought that the name `MemoryConsumer` might not make sense if it was used by places that can't spill, but I suppose that those places could just have `spill()` return 0. So I'm fine sticking with the current name.


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

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


[GitHub] spark pull request: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151683415
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44474/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152098576
  
    **[Test build #44579 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44579/consoleFull)** for PR 9241 at commit [`7bf76e5`](https://github.com/apache/spark/commit/7bf76e5a29b49c97fd2d2e22ba244e2dba74df51).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151725760
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151992609
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152034271
  
    **[Test build #44542 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44542/consoleFull)** for PR 9241 at commit [`827d4f0`](https://github.com/apache/spark/commit/827d4f0a6d42e4f2af37eedacf3098fceb896c32).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43443676
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public abstract class MemoryConsumer {
    +
    +  private TaskMemoryManager memoryManager;
    --- End diff --
    
    Do you mind renaming this variable to `taskMemoryManager` in order to avoid any ambiguity?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151683414
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43464962
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -678,20 +747,31 @@ private void allocate(int capacity) {
       }
     
       /**
    +   * Free the memory used by longArray.
    +   */
    +  public void freeArray() {
    --- End diff --
    
    This also nulls out the bitset, but i guess we don't do memory accounting for it since it's so relatively small?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152085212
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43448855
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -108,42 +102,20 @@ public ShuffleExternalSorter(
           int initialSize,
           int numPartitions,
           SparkConf conf,
    -      ShuffleWriteMetrics writeMetrics) throws IOException {
    +      ShuffleWriteMetrics writeMetrics) {
    +    super(memoryManager);
         this.taskMemoryManager = memoryManager;
         this.blockManager = blockManager;
         this.taskContext = taskContext;
    -    this.initialSize = initialSize;
         this.peakMemoryUsedBytes = initialSize;
    --- End diff --
    
    I think that this is a bug in the old code as well, but I'm pretty sure that this initial `peakMemoryUsedBytes` is wrong: `initialiSize` should actually be called `initialNumElements` or `initialCapacity` or something, since it's measured in terms of number of entires, not 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151674588
  
    **[Test build #44474 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44474/consoleFull)** for PR 9241 at commit [`7087f2f`](https://github.com/apache/spark/commit/7087f2f4ca9841716b9b4740c037b410607cf719).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152443044
  
    Great, sounds good to me! I do think that this patch might benefit from more comments to explain the code and I'd be happy to help with that in a followup 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151726487
  
    **[Test build #44491 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44491/consoleFull)** for PR 9241 at commit [`ce24f03`](https://github.com/apache/spark/commit/ce24f039f07bde78500191a9afec7a6298a6b559).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43457537
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public abstract class MemoryConsumer {
    +
    +  private final TaskMemoryManager taskMemoryManager;
    +  private final long pageSize;
    +  private long used;
    +
    +  protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize) {
    +    this.taskMemoryManager = taskMemoryManager;
    +    if (pageSize == 0) {
    +      pageSize = taskMemoryManager.pageSizeBytes();
    +    }
    +    this.pageSize = pageSize;
    +    this.used = 0;
    +  }
    +
    +  protected MemoryConsumer(TaskMemoryManager taskMemoryManager) {
    +    this(taskMemoryManager, taskMemoryManager.pageSizeBytes());
    +  }
    +
    +  /**
    +   * Returns the size of used memory in bytes.
    +   */
    +  long getUsed() {
    +    return used;
    +  }
    +
    +  /**
    +   * Force spill during building.
    +   *
    +   * For testing.
    +   */
    +  public void spill() throws IOException {
    +    spill(Long.MAX_VALUE, this);
    +  }
    +
    +  /**
    +   * Spill some data to disk to release memory, which will be called by TaskMemoryManager
    +   * when there is not enough memory for the task.
    +   *
    +   * This should be implemented by subclass.
    +   *
    +   * Note: In order to avoid possible deadlock, should not call acquireMemory() from spill().
    +   *
    +   * @param size the amount of memory should be released
    +   * @param trigger the MemoryConsumer that trigger this spilling
    +   * @return the amount of released memory in bytes
    +   * @throws IOException
    +   */
    +  public abstract long spill(long size, MemoryConsumer trigger) throws IOException;
    +
    +  /**
    +   * Acquire `size` bytes memory.
    +   *
    +   * If there is not enough memory, throws OutOfMemoryError.
    +   */
    +  protected void acquireMemory(long size) {
    +    long got = taskMemoryManager.acquireExecutionMemory(size, this);
    +    if (got < size) {
    +      taskMemoryManager.releaseExecutionMemory(got, this);
    +      taskMemoryManager.showMemoryUsage();
    +      throw new OutOfMemoryError("Could not acquire " + size + " bytes of memory, got " + got);
    +    }
    +    used += got;
    +  }
    +
    +  /**
    +   * Release `size` bytes memory.
    +   */
    +  protected void releaseMemory(long size) {
    +    taskMemoryManager.releaseExecutionMemory(size, this);
    +    used -= size;
    +  }
    +
    +  /**
    +   * Allocate a memory block with at least `required` bytes.
    +   *
    +   * Throws IOException if there is not enough memory.
    +   *
    +   * @throws OutOfMemoryError
    +   */
    +  protected MemoryBlock allocatePage(long required) {
    +    MemoryBlock page = taskMemoryManager.allocatePage(Math.max(pageSize, required), this);
    +    if (page == null || page.size() < required) {
    +      long got = 0;
    +      if (page != null) {
    +        got = page.size();
    +        freePage(page);
    +      }
    +      taskMemoryManager.showMemoryUsage();
    +      throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
    +    }
    +    used += page.size();
    +    return page;
    +  }
    +
    +  /**
    +   * Free a memory block.
    +   */
    +  protected void freePage(MemoryBlock page) {
    +    taskMemoryManager.freePage(page, this);
    +    used -= page.size();
    --- End diff --
    
    Again, maybe an invalid concern, but is it safe to call `page.size()` on a freed page?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458200
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +105,103 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashSet<MemoryConsumer> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashSet<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers) {
    --- End diff --
    
    No, we never remove it, and it will not add more under this lock.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43454317
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords--;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    +        ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
    +
    +        long released = 0L;
    +        while (dataPages.size() > 0) {
    +          MemoryBlock block = dataPages.getLast();
    +          // The currentPage is used, cannot be released
    +          if (block == currentPage) {
    +            break;
    +          }
    +
    +          Object base = block.getBaseObject();
    +          long offset = block.getBaseOffset();
    +          int numRecords = Platform.getInt(base, offset);
    +          offset += 4;
    +          final UnsafeSorterSpillWriter writer =
    +            new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords);
    +          while (numRecords-- > 0) {
    +            int length = Platform.getInt(base, offset);
    +            writer.write(base, offset + 4, length, 0);
    +            offset += 4 + length;
    +          }
    +          writer.close();
    +          spillWriters.add(writer);
    +          if (TaskContext.get() != null) {
    +            TaskContext.get().addOnCompleteCallback(new AbstractFunction0<BoxedUnit>() {
    --- End diff --
    
    It turns out that `addOnCompleteCallback()` has been deprecated, so I would use `addTaskCompletionListener` instead (which is also more friendly to implementation in Java).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151581415
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152299162
  
    **[Test build #44613 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44613/consoleFull)** for PR 9241 at commit [`51278f8`](https://github.com/apache/spark/commit/51278f89d73d46f8cf22377c3bd0b7aa3ad5c02e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150999439
  
    Hey @davies,
    
    This patch makes sense to me at a high-level, but I have a few questions:
    
    - Could you add a description to this pull request and share your list of remaining TODOs with me (maybe by posting it as a checklist in the PR description)?
    - Can you comment on the thread-safety concerns here? My current understanding is that we don't have to worry about memory-manager-triggered spills racing with other interactions on the spillables because the iterator model provides some implicit synchronization. In order to convince ourselves that this is safe in all cases, however, I'd like to think through two corner-cases:
      - What happens if a single task contains multiple threads? Currently, this can happen in PythonRDD, PipedRDD, and a couple of other places. All of these cases are situations where we have a writer or reader thread for interacting with an external process. Although we have separate threads, they are somewhat synchronous / coupled via their interaction with the external process. This could be tricky, though, so I'd like to talk through some examples to make sure we've covered all of the tricky cases.
      - What happens if operator B is in the middle of processing a `next()` call on its iterator, which calls it's parent's `next()` method, which requires memory to grow, which triggers a spill that drains memory from A and de-allocates or spills data structures that it's relying on? Do we have to make any distinctions between asking an upstream operator to spill versus a downstream one?
      - Are there any risks of deadlocks with the extra synchronization added here?
    
    I'm going to focus on merging my memory manager unification patch tonight so that we can start rebasing this tomorrow.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151657419
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43318002
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public abstract class MemoryConsumer {
    --- End diff --
    
    Is it too long?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152436956
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43449556
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -108,42 +102,20 @@ public ShuffleExternalSorter(
           int initialSize,
           int numPartitions,
           SparkConf conf,
    -      ShuffleWriteMetrics writeMetrics) throws IOException {
    +      ShuffleWriteMetrics writeMetrics) {
    +    super(memoryManager);
         this.taskMemoryManager = memoryManager;
         this.blockManager = blockManager;
         this.taskContext = taskContext;
    -    this.initialSize = initialSize;
         this.peakMemoryUsedBytes = initialSize;
         this.numPartitions = numPartitions;
         // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided
         this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
         this.numElementsForSpillThreshold =
           conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MAX_VALUE);
    -    this.pageSizeBytes = (int) Math.min(
    -      PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, taskMemoryManager.pageSizeBytes());
    --- End diff --
    
    In other words, `allocatePage()` needs to let us impose a maximum size on the returned page in case we need to allocate a page that's smaller than a jumbo page size.
    
    Of course, if the default page size can't be bigger than `PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES` then we're 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 pull request: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150491947
  
    **[Test build #44210 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44210/consoleFull)** for PR 9241 at commit [`3562476`](https://github.com/apache/spark/commit/35624760abe0a1565590d94e9b6ff6746dfaed26).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151773750
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43460497
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java ---
    @@ -83,37 +83,43 @@ public UnsafeInMemorySorter(
           final PrefixComparator prefixComparator,
           int initialSize) {
         assert (initialSize > 0);
    -    this.pointerArray = new long[initialSize * 2];
    +    this.array = new long[initialSize * 2];
         this.memoryManager = memoryManager;
         this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE);
         this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager);
       }
     
    +  public void reset() {
    +    pos = 0;
    +  }
    +
       /**
        * @return the number of records that have been inserted into this sorter.
        */
       public int numRecords() {
    -    return pointerArrayInsertPosition / 2;
    +    return pos / 2;
       }
     
    -  public long getMemoryUsage() {
    -    return pointerArray.length * 8L;
    +  private int newLength() {
    +    return array.length < Integer.MAX_VALUE / 2 ? (array.length * 2) : Integer.MAX_VALUE;
    +  }
    +
    +  public long getMemoryToExpand() {
    +    return (long) (newLength() - array.length) * 8L;
    --- End diff --
    
    We already acquire the whole memory for new array first, then release the memory for old 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 pull request: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152306265
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43311277
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,97 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got, consumer);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got, consumer);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    --- End diff --
    
    what if consumer is the only one in consumers ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151673397
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152286880
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152399627
  
    **[Test build #44652 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44652/consoleFull)** for PR 9241 at commit [`4ee1f42`](https://github.com/apache/spark/commit/4ee1f42b08b46469d8631a56b01898d251ebf3e7).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43449494
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // consumers could be modified by spill(), so we should have a copy here.
    +        MemoryConsumer[] cs = new MemoryConsumer[consumers.size()];
    +        consumers.keySet().toArray(cs);
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: cs) {
    +          if (c != null && c != consumer) {
    +            try {
    +              long released = c.spill(required - got, consumer);
    +              if (released > 0) {
    +                logger.info("Task {} released {} from {} for {}", taskAttemptId,
    +                  Utils.bytesToString(released), c, consumer);
    +                got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +                if (got >= required) {
    +                  break;
    +                }
    +              }
    +            } catch (IOException e) {
    +              logger.error("error while calling spill() on " + c, e);
    +              throw new OutOfMemoryError("error while calling spill() on " + c + " : "
    +                + e.getMessage());
    --- End diff --
    
    It does not support that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152296921
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150739095
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44265/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43449223
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -108,42 +102,20 @@ public ShuffleExternalSorter(
           int initialSize,
           int numPartitions,
           SparkConf conf,
    -      ShuffleWriteMetrics writeMetrics) throws IOException {
    +      ShuffleWriteMetrics writeMetrics) {
    +    super(memoryManager);
         this.taskMemoryManager = memoryManager;
         this.blockManager = blockManager;
         this.taskContext = taskContext;
    -    this.initialSize = initialSize;
         this.peakMemoryUsedBytes = initialSize;
         this.numPartitions = numPartitions;
         // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided
         this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
         this.numElementsForSpillThreshold =
           conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MAX_VALUE);
    -    this.pageSizeBytes = (int) Math.min(
    -      PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, taskMemoryManager.pageSizeBytes());
    --- End diff --
    
    Is it safe to make this change? After this change, `PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES` is only used in test code. The packed record pointer is used to compress pointers to addresses inside of memory pages (for use in the optimized sorter here) and thus imposes a limit on the size of pages that contain multiple sorted records. If we omit this check then we might run into corruption or other issues if `taskMemoryManager.pageSizeBytes()` is bigger than `PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES`. This is only likely to be an issue on machines with huge heap sizes, but I still think we should guard against 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152287762
  
    **[Test build #44625 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44625/consoleFull)** for PR 9241 at commit [`27ff4fc`](https://github.com/apache/spark/commit/27ff4fc859b40afdad379068a6ac0d06f9c09eef).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43442582
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    --- End diff --
    
    This could potentially simplify some of the code in the transferMemory and releaseMemory methods down 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152126730
  
    **[Test build #44579 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44579/consoleFull)** for PR 9241 at commit [`7bf76e5`](https://github.com/apache/spark/commit/7bf76e5a29b49c97fd2d2e22ba244e2dba74df51).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151574813
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44447/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152110562
  
    **[Test build #1960 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1960/consoleFull)** for PR 9241 at commit [`0971d27`](https://github.com/apache/spark/commit/0971d274de4b642a43b2898c8f535f9721914dcb).
     * 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43464654
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +230,158 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    --- End diff --
    
    Ah, because we rely on `reader.hasNext()` when we're dealing with on-disk data.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151630056
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44453/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43561746
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +106,104 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  @GuardedBy("this")
    +  private final HashSet<MemoryConsumer> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashSet<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers) {
    +          if (c != null && c != consumer && c.getUsed() > 0) {
    +            try {
    +              long released = c.spill(required - got, consumer);
    +              if (released > 0) {
    +                logger.info("Task {} released {} from {} for {}", taskAttemptId,
    +                  Utils.bytesToString(released), c, consumer);
    +                got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +                if (got >= required) {
    +                  break;
    +                }
    +              }
    +            } catch (IOException e) {
    --- End diff --
    
    Should this catch clause be moved to wrap c.spill() at line 142 ?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151657345
  
    **[Test build #44450 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44450/consoleFull)** for PR 9241 at commit [`ee6b9a4`](https://github.com/apache/spark/commit/ee6b9a40d01b0ae9d43e34511c43c77f8a080bf0).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152254245
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43452194
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    --- End diff --
    
    Could there be multiple spills to clean up 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151701043
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152299245
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152306301
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953350
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala ---
    @@ -128,17 +161,31 @@ class ShuffleMemoryManager protected (
           metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq)
         }
         taskMemory(taskAttemptId) += acquired
    +    val consumers = taskConsumers.getOrElseUpdate(taskAttemptId, new mutable.HashMap())
    +    consumers += consumer -> (consumers.getOrElse(consumer, 0L) + acquired)
         acquired
       }
     
    +  def release(numBytes: Long): Unit = release(numBytes, null)
    +
       /** Release numBytes bytes for the current task. */
    -  def release(numBytes: Long): Unit = memoryManager.synchronized {
    +  def release(numBytes: Long, consumer: MemoryConsumer): Unit = memoryManager.synchronized {
         val taskAttemptId = currentTaskAttemptId()
         val curMem = taskMemory.getOrElse(taskAttemptId, 0L)
         if (curMem < numBytes) {
           throw new SparkException(
             s"Internal error: release called on $numBytes bytes but task only has $curMem")
         }
    +    taskConsumers.get(taskAttemptId).map { consumers =>
    +      if (consumers.contains(consumer)) {
    +        consumers(consumer) -= numBytes
    +        if (consumers(consumer) <= 0) {
    +          consumers.remove(consumer)
    +        }
    +      } else {
    +        logError(s"BUG: release $numBytes from $consumer in task $taskAttemptId")
    --- End diff --
    
    If the `spark.testing` system property is set, then I think this warning should become an exception, since that will let us detect problems in unit 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 pull request: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152308300
  
    **[Test build #44631 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44631/consoleFull)** for PR 9241 at commit [`c044afe`](https://github.com/apache/spark/commit/c044afe62a36d98934d4e9f36665e2a4837783c7).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43459132
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java ---
    @@ -37,33 +37,52 @@ public int compare(PackedRecordPointer left, PackedRecordPointer right) {
        * {@link PackedRecordPointer}. The sort operates on this array instead of directly manipulating
        * records.
        */
    -  private long[] pointerArray;
    +  private long[] array;
     
       /**
        * The position in the pointer array where new records can be inserted.
        */
    -  private int pointerArrayInsertPosition = 0;
    +  private int pos = 0;
     
       public ShuffleInMemorySorter(int initialSize) {
         assert (initialSize > 0);
    -    this.pointerArray = new long[initialSize];
    -    this.sorter = new Sorter<PackedRecordPointer, long[]>(ShuffleSortDataFormat.INSTANCE);
    +    this.array = new long[initialSize];
    +    this.sorter = new Sorter<>(ShuffleSortDataFormat.INSTANCE);
       }
     
    -  public void expandPointerArray() {
    -    final long[] oldArray = pointerArray;
    +  public int numRecords() {
    +    return pos;
    +  }
    +
    +  public void reset() {
    +    pos = 0;
    +  }
    +
    +  private int newLength() {
         // Guard against overflow:
    -    final int newLength = oldArray.length * 2 > 0 ? (oldArray.length * 2) : Integer.MAX_VALUE;
    -    pointerArray = new long[newLength];
    -    System.arraycopy(oldArray, 0, pointerArray, 0, oldArray.length);
    +    return array.length <= Integer.MAX_VALUE / 2 ?
    +      (array.length * 2) : Integer.MAX_VALUE;
    --- End diff --
    
    Don't necessarily have to wrap 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150994599
  
    I know that this is still `WIP`, but were you thinking about also enabling this for the two `Spillable` collections (ExternalAppendOnlyMap and ExternalSorter)? That's probably a lower priority given that we're most concerned about optimizing SQL's memory usage, but it would still be nice to do. If we decide to defer it for now, let's create a followup task to do it in the next 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 pull request: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150704549
  
    **[Test build #1948 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1948/consoleFull)** for PR 9241 at commit [`5c198cf`](https://github.com/apache/spark/commit/5c198cf329748a0af9f108d182615616334a6c6a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152112891
  
    **[Test build #44572 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44572/consoleFull)** for PR 9241 at commit [`a3e01d0`](https://github.com/apache/spark/commit/a3e01d0c9222bee4bc8eb48e2542a4344e72aa33).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152346334
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458249
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -108,42 +102,20 @@ public ShuffleExternalSorter(
           int initialSize,
           int numPartitions,
           SparkConf conf,
    -      ShuffleWriteMetrics writeMetrics) throws IOException {
    +      ShuffleWriteMetrics writeMetrics) {
    +    super(memoryManager);
         this.taskMemoryManager = memoryManager;
         this.blockManager = blockManager;
         this.taskContext = taskContext;
    -    this.initialSize = initialSize;
         this.peakMemoryUsedBytes = initialSize;
         this.numPartitions = numPartitions;
         // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided
         this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
         this.numElementsForSpillThreshold =
           conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MAX_VALUE);
    -    this.pageSizeBytes = (int) Math.min(
    -      PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, taskMemoryManager.pageSizeBytes());
    --- End diff --
    
    Added it back.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151586418
  
    **[Test build #44453 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44453/consoleFull)** for PR 9241 at commit [`49b8135`](https://github.com/apache/spark/commit/49b8135bf37347fb057f363398c17367f0f0c991).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152112976
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44572/
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151585798
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150739092
  
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151755297
  
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953325
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala ---
    @@ -90,33 +101,55 @@ class ShuffleMemoryManager protected (
     
           // How much we can grant this task; don't let it grow to more than 1 / numActiveTasks;
           // don't let it be negative
    -      val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveTasks) - curMem))
    -      // Only give it as much memory as is free, which might be none if it reached 1 / numTasks
    -      val toGrant = math.min(maxToGrant, freeMemory)
    -
    -      if (curMem < maxMemory / (2 * numActiveTasks)) {
    -        // We want to let each task get at least 1 / (2 * numActiveTasks) before blocking;
    -        // if we can't give it this much now, wait for other tasks to free up memory
    -        // (this happens if older tasks allocated lots of memory before N grew)
    -        if (freeMemory >= math.min(maxToGrant, maxMemory / (2 * numActiveTasks) - curMem)) {
    -          return acquire(toGrant)
    +      // Only give it as much memory as is 1/4 of free, which might be none
    +      // if it reached 1 / numTasks
    +      val available = math.min(freeMemory, math.max(0, (maxMemory / numActiveTasks) - curMem))
    +      if (available >= numBytes) {
    +        return acquire(consumer, numBytes)
    +      }
    +      val needed = numBytes - available
    +      if (consumer != null && consumer.release(needed) >= needed) {
    +        return acquire(consumer, numBytes)
    +      }
    +      if (!released && taskConsumers.contains(taskAttemptId)) {
    +        // try to release more to make sure that we will got enough memory in next loop
    +        tryRelease(numBytes - available)
    +        released = true
    +      } else {
    +        if (curMem < maxMemory / (2 * numActiveTasks)) {
    +          return acquire(consumer, available)
             } else {
    -          logInfo(
    -            s"TID $taskAttemptId waiting for at least 1/2N of shuffle memory pool to be free")
    +          // in case of memory is not balanced, try to protected the ones already have more memory
    +          // wait for other tasks to finished or fail (release memory)
               memoryManager.wait()
             }
    -      } else {
    -        return acquire(toGrant)
           }
         }
         0L  // Never reached
       }
     
    +  private def tryRelease(numBytes: Long): Long = {
    +    val taskAttemptId = currentTaskAttemptId()
    +    var released = 0L
    +    taskConsumers(taskAttemptId).foreach { case (consumer, used) =>
    +      if (consumer != null && used > 0) {
    +        released += consumer.release(numBytes - released)
    +        if (released >= numBytes) {
    +          return released
    +        }
    +      }
    +    }
    +    released
    +  }
    +
       /**
        * Acquire N bytes of execution memory from the memory manager for the current task.
        * @return number of bytes actually acquired (<= N).
        */
    -  private def acquire(numBytes: Long): Long = memoryManager.synchronized {
    +  private def acquire(consumer: MemoryConsumer, numBytes: Long): Long = memoryManager.synchronized {
    +    if (numBytes <= 0) {
    --- End diff --
    
    Should we add an assert and guard against requesting a negative amount of memory, since that probably indicates a bug elsewhere?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151657424
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44450/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458974
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java ---
    @@ -295,8 +290,22 @@ public void cleanupResources() {
       private void growPointerArrayIfNecessary() throws IOException {
         assert(inMemSorter != null);
         if (!inMemSorter.hasSpaceForAnotherRecord()) {
    -      // TODO: track the pointer array memory! (SPARK-10474)
    -      inMemSorter.expandPointerArray();
    +      long used = inMemSorter.getMemoryUsage();
    +      long needed = inMemSorter.getMemoryToExpand();
    +      try {
    +        acquireMemory(used + needed);  // could trigger spilling
    +        if (inMemSorter.hasSpaceForAnotherRecord()) {
    --- End diff --
    
    Isn't it guaranteed that `inMemSorter.hasSpaceForAnotherRecord() == false` due to the outer `if` condition 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 pull request: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953398
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -227,62 +238,147 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class BytesToBytesMapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters =
    +      new LinkedList<UnsafeSorterSpillWriter>();
    +    private UnsafeSorterSpillReader reader = null;
    +
    +    private BytesToBytesMapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    -      }
    +      destructiveIterator = this;
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    -        this.bmap.shuffleMemoryManager.release(currentPage.size());
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          taskMemoryManager.freePage(currentPage);
    +          shuffleMemoryManager.release(currentPage.size());
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          try {
    +            reader = spillWriters.removeFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords --;
    --- End diff --
    
    Mind omitting the space between `numRecords` and `--`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152403185
  
    **[Test build #44656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44656/consoleFull)** for PR 9241 at commit [`e943e74`](https://github.com/apache/spark/commit/e943e745317bd4d13934a14026be5a7915706269).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43443333
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // consumers could be modified by spill(), so we should have a copy here.
    +        MemoryConsumer[] cs = new MemoryConsumer[consumers.size()];
    +        consumers.keySet().toArray(cs);
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: cs) {
    +          if (c != null && c != consumer) {
    +            try {
    +              long released = c.spill(required - got, consumer);
    +              if (released > 0) {
    +                logger.info("Task {} released {} from {} for {}", taskAttemptId,
    +                  Utils.bytesToString(released), c, consumer);
    +                got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +                if (got >= required) {
    +                  break;
    +                }
    +              }
    +            } catch (IOException e) {
    +              logger.error("error while calling spill() on " + c, e);
    +              throw new OutOfMemoryError("error while calling spill() on " + c + " : "
    +                + e.getMessage());
    +            }
    +          }
    +        }
    +      }
    +
    +      // call spill() on itself
    +      if (got < required && consumer != null) {
    +        try {
    +          long released = consumer.spill(required - got, consumer);
    +          if (released > 0) {
    +            logger.info("Task {} released {} from itself ({})", taskAttemptId,
    +              Utils.bytesToString(released), consumer);
    +            got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +          }
    +        } catch (IOException e) {
    +          logger.error("error while calling spill() on " + consumer, e);
    +          throw new OutOfMemoryError("error while calling spill() on " + consumer + " : "
    +            + e.getMessage());
    +        }
    +      }
    +
    +      // Update the accounting, even consumer is null
    +      if (got > 0) {
    +        long old = 0L;
    +        if (consumers.containsKey(consumer)) {
    +          old = consumers.get(consumer);
    +        }
    +        consumers.put(consumer, got + old);
    +      }
    +
    +      logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer);
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    +    assert(size >= 0);
    +    if (size == 0) {
    +      return;
    +    }
    +    synchronized (this) {
    +      if (consumers.containsKey(consumer)) {
    +        long old = consumers.get(consumer);
    +        if (old > size) {
    +          consumers.put(consumer, old - size);
    +        } else {
    +          if (old < size) {
    +            String msg = "Release " + size + " bytes memory (more than acquired " + old + ") for "
    +              + consumer;
    +            logger.warn(msg);
    +            if (Utils.isTesting()) {
    +              Platform.throwException(new SparkException(msg));
    +            }
    +          }
    +          consumers.remove(consumer);
    +        }
    +      } else {
    +        String msg = "Release " + size + " bytes memory for non-existent " + consumer;
    +        logger.warn(msg);
    +        if (Utils.isTesting()) {
    +          Platform.throwException(new SparkException(msg));
    +        }
    +      }
    +    }
    +
    +    logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer);
         memoryManager.releaseExecutionMemory(size, taskAttemptId);
       }
     
    +  public void transferOwnership(long size, MemoryConsumer from, MemoryConsumer to) {
    --- End diff --
    
    It looks like this method isn't used anywhere?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151574765
  
    **[Test build #44447 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44447/consoleFull)** for PR 9241 at commit [`d0ada7b`](https://github.com/apache/spark/commit/d0ada7b5c6f8be9a91624e4eb5a6ca2242edf16b).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151773751
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44504/
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151689479
  
    **[Test build #44481 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44481/consoleFull)** for PR 9241 at commit [`97935e5`](https://github.com/apache/spark/commit/97935e560453dcfa0e4cf98ab0c54da39d6b612a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152333075
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44625/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152338548
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44629/
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#discussion_r43188878
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,100 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public class MemoryConsumer {
    --- End diff --
    
    Is the idea that each operator will have its own subclass of MemoryConsumer which implements `spill()`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152399804
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44652/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152089228
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150651755
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152085503
  
    **[Test build #44569 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44569/consoleFull)** for PR 9241 at commit [`0971d27`](https://github.com/apache/spark/commit/0971d274de4b642a43b2898c8f535f9721914dcb).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43318511
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,97 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got, consumer);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got, consumer);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    --- End diff --
    
    My point was that got may not be equal to size coming out of the loop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953597
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -658,20 +707,28 @@ public boolean putNewKey(
        * Acquire a new page from the {@link ShuffleMemoryManager}.
        * @return whether there is enough space to allocate the new page.
        */
    -  private boolean acquireNewPage() {
    -    final long memoryGranted = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
    -    if (memoryGranted != pageSizeBytes) {
    -      shuffleMemoryManager.release(memoryGranted);
    -      logger.debug("Failed to acquire {} bytes of memory", pageSizeBytes);
    +  private boolean acquireNewPage(long numBytes) {
    +    long granted = shuffleMemoryManager.tryToAcquire(Math.max(numBytes, pageSizeBytes), this);
    +    if (granted < numBytes) {
    +      shuffleMemoryManager.release(granted, this);
    +      logger.debug("Failed to acquire {} bytes of memory", numBytes);
           return false;
         }
    -    MemoryBlock newPage = taskMemoryManager.allocatePage(pageSizeBytes);
    +    MemoryBlock newPage = taskMemoryManager.allocatePage(granted);
         dataPages.add(newPage);
    -    pageCursor = 0;
    -    currentDataPage = newPage;
    +    Platform.putInt(newPage.getBaseObject(), newPage.getBaseOffset(), 0);
    +    pageCursor = 4;
    +    currentPage = newPage;
         return true;
       }
     
    +  public long release(long numBytes) throws IOException {
    --- End diff --
    
    `@Override`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43319523
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,97 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got, consumer);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got, consumer);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    --- End diff --
    
    This function does not garantee that `got` will be equal to `size`, after call spill().


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43451586
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    +      currentPage = allocatePage(required);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
        * Write a record to the shuffle sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      int partitionId) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId)
    +    throws IOException {
     
    -    if (numRecordsInsertedSinceLastSpill > numElementsForSpillThreshold) {
    +    // for tests
    +    assert(inMemSorter != null);
    +    if (inMemSorter.numRecords() > numElementsForSpillThreshold) {
           spill();
         }
     
         growPointerArrayIfNecessary();
    -    // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    --- End diff --
    
    Sure; just add a JIRA subtask so we don't forget.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152379629
  
    LGTM overall right now (pending Jenkins for commit 4ee1f42). We can address any minor issues in followups.
    
    I'll take one final glance at one of the new unit tests when I get home, then will merge this to unblock my 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 pull request: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152370890
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152370910
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152098120
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152360279
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43453011
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords--;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    +        ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
    +
    +        long released = 0L;
    +        while (dataPages.size() > 0) {
    +          MemoryBlock block = dataPages.getLast();
    +          // The currentPage is used, cannot be released
    +          if (block == currentPage) {
    +            break;
    +          }
    +
    +          Object base = block.getBaseObject();
    +          long offset = block.getBaseOffset();
    +          int numRecords = Platform.getInt(base, offset);
    +          offset += 4;
    +          final UnsafeSorterSpillWriter writer =
    +            new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords);
    +          while (numRecords-- > 0) {
    +            int length = Platform.getInt(base, offset);
    +            writer.write(base, offset + 4, length, 0);
    +            offset += 4 + length;
    +          }
    +          writer.close();
    +          spillWriters.add(writer);
    --- End diff --
    
    I guess that the order of iteration doesn't really matter for BytesToBytesMap / isn't specified, so it doesn't really matter whether we preserve ordering of the pages after spilling.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43444410
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala ---
    @@ -193,10 +193,6 @@ class UnsafeFixedWidthAggregationMapSuite
         // Convert the map into a sorter
         val sorter = map.destructAndCreateExternalSorter()
     
    -    withClue(s"destructAndCreateExternalSorter should release memory used by the map") {
    --- End diff --
    
    Was this assertion invalid before? Or has this changed (are we now tracking pointer array memory usage)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152332873
  
    **[Test build #44625 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44625/consoleFull)** for PR 9241 at commit [`27ff4fc`](https://github.com/apache/spark/commit/27ff4fc859b40afdad379068a6ac0d06f9c09eef).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150491511
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151629928
  
    **[Test build #44453 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44453/consoleFull)** for PR 9241 at commit [`49b8135`](https://github.com/apache/spark/commit/49b8135bf37347fb057f363398c17367f0f0c991).
     * 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43461576
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -531,113 +645,59 @@ public int getValueLength() {
          * @return true if the put() was successful and false if the put() failed because memory could
          *         not be acquired.
          */
    -    public boolean putNewKey(
    -        Object keyBaseObject,
    -        long keyBaseOffset,
    -        int keyLengthBytes,
    -        Object valueBaseObject,
    -        long valueBaseOffset,
    -        int valueLengthBytes) {
    +    public boolean putNewKey(Object keyBase, long keyOffset, int keyLength,
    +        Object valueBase, long valueOffset, int valueLength) {
           assert (!isDefined) : "Can only set value once for a key";
    -      assert (keyLengthBytes % 8 == 0);
    -      assert (valueLengthBytes % 8 == 0);
    +      assert (keyLength % 8 == 0);
    +      assert (valueLength % 8 == 0);
           assert(bitset != null);
           assert(longArray != null);
     
    -      if (numElements == MAX_CAPACITY) {
    -        throw new IllegalStateException("BytesToBytesMap has reached maximum capacity");
    +      if (numElements == MAX_CAPACITY || !canGrowArray) {
    +        return false;
           }
     
           // Here, we'll copy the data into our data pages. Because we only store a relative offset from
           // the key address instead of storing the absolute address of the value, the key and value
           // must be stored in the same memory page.
           // (8 byte key length) (key) (value)
    -      final long requiredSize = 8 + keyLengthBytes + valueLengthBytes;
    -
    -      // --- Figure out where to insert the new record ---------------------------------------------
    -
    -      final MemoryBlock dataPage;
    -      final Object dataPageBaseObject;
    -      final long dataPageInsertOffset;
    -      boolean useOverflowPage = requiredSize > pageSizeBytes - 8;
    -      if (useOverflowPage) {
    -        // The record is larger than the page size, so allocate a special overflow page just to hold
    -        // that record.
    -        final long overflowPageSize = requiredSize + 8;
    -        MemoryBlock overflowPage = taskMemoryManager.allocatePage(overflowPageSize);
    -        if (overflowPage == null) {
    -          logger.debug("Failed to acquire {} bytes of memory", overflowPageSize);
    -          return false;
    -        }
    -        dataPages.add(overflowPage);
    -        dataPage = overflowPage;
    -        dataPageBaseObject = overflowPage.getBaseObject();
    -        dataPageInsertOffset = overflowPage.getBaseOffset();
    -      } else if (currentDataPage == null || pageSizeBytes - 8 - pageCursor < requiredSize) {
    -        // The record can fit in a data page, but either we have not allocated any pages yet or
    -        // the current page does not have enough space.
    -        if (currentDataPage != null) {
    -          // There wasn't enough space in the current page, so write an end-of-page marker:
    -          final Object pageBaseObject = currentDataPage.getBaseObject();
    -          final long lengthOffsetInPage = currentDataPage.getBaseOffset() + pageCursor;
    -          Platform.putInt(pageBaseObject, lengthOffsetInPage, END_OF_PAGE_MARKER);
    -        }
    -        if (!acquireNewPage()) {
    +      final long recordLength = 8 + keyLength + valueLength;
    +      if (currentPage == null || currentPage.size() - pageCursor < recordLength) {
    +        if (!acquireNewPage(recordLength + 4L)) {
               return false;
             }
    -        dataPage = currentDataPage;
    -        dataPageBaseObject = currentDataPage.getBaseObject();
    -        dataPageInsertOffset = currentDataPage.getBaseOffset();
    -      } else {
    -        // There is enough space in the current data page.
    -        dataPage = currentDataPage;
    -        dataPageBaseObject = currentDataPage.getBaseObject();
    -        dataPageInsertOffset = currentDataPage.getBaseOffset() + pageCursor;
           }
     
           // --- Append the key and value data to the current data page --------------------------------
    -
    -      long insertCursor = dataPageInsertOffset;
    -
    -      // Compute all of our offsets up-front:
    -      final long recordOffset = insertCursor;
    -      insertCursor += 4;
    -      final long keyLengthOffset = insertCursor;
    -      insertCursor += 4;
    -      final long keyDataOffsetInPage = insertCursor;
    -      insertCursor += keyLengthBytes;
    -      final long valueDataOffsetInPage = insertCursor;
    -      insertCursor += valueLengthBytes; // word used to store the value size
    -
    -      Platform.putInt(dataPageBaseObject, recordOffset,
    -        keyLengthBytes + valueLengthBytes + 4);
    -      Platform.putInt(dataPageBaseObject, keyLengthOffset, keyLengthBytes);
    -      // Copy the key
    -      Platform.copyMemory(
    -        keyBaseObject, keyBaseOffset, dataPageBaseObject, keyDataOffsetInPage, keyLengthBytes);
    -      // Copy the value
    -      Platform.copyMemory(valueBaseObject, valueBaseOffset, dataPageBaseObject,
    -        valueDataOffsetInPage, valueLengthBytes);
    -
    -      // --- Update bookeeping data structures -----------------------------------------------------
    -
    -      if (useOverflowPage) {
    -        // Store the end-of-page marker at the end of the data page
    -        Platform.putInt(dataPageBaseObject, insertCursor, END_OF_PAGE_MARKER);
    -      } else {
    -        pageCursor += requiredSize;
    -      }
    -
    +      final Object base = currentPage.getBaseObject();
    +      long offset = currentPage.getBaseOffset() + pageCursor;
    +      final long recordOffset = offset;
    +      Platform.putInt(base, offset, keyLength + valueLength + 4);
    +      Platform.putInt(base, offset + 4, keyLength);
    +      offset += 8;
    +      Platform.copyMemory(keyBase, keyOffset, base, offset, keyLength);
    +      offset += keyLength;
    +      Platform.copyMemory(valueBase, valueOffset, base, offset, valueLength);
    +
    +      // --- Update bookkeeping data structures -----------------------------------------------------
    +      offset = currentPage.getBaseOffset();
    +      Platform.putInt(base, offset, Platform.getInt(base, offset) + 1);
    +      pageCursor += recordLength;
           numElements++;
           bitset.set(pos);
           final long storedKeyAddress = taskMemoryManager.encodePageNumberAndOffset(
    -        dataPage, recordOffset);
    +        currentPage, recordOffset);
           longArray.set(pos * 2, storedKeyAddress);
           longArray.set(pos * 2 + 1, keyHashcode);
           updateAddressesAndSizes(storedKeyAddress);
           isDefined = true;
    +
           if (numElements > growthThreshold && longArray.size() < MAX_CAPACITY) {
    -        growAndRehash();
    +        try {
    +          growAndRehash();
    +        } catch (OutOfMemoryError oom) {
    --- End diff --
    
    In other words, I think that we need to make sure that any operation where we're catching OOM is atomic no matter where OOM occurs./


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152089469
  
    **[Test build #1960 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1960/consoleFull)** for PR 9241 at commit [`0971d27`](https://github.com/apache/spark/commit/0971d274de4b642a43b2898c8f535f9721914dcb).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152353141
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44631/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43452944
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    --- End diff --
    
    Oh no, there will be multiple file, one for each page.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152385979
  
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150701783
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151735357
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151584124
  
    **[Test build #44450 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44450/consoleFull)** for PR 9241 at commit [`ee6b9a4`](https://github.com/apache/spark/commit/ee6b9a40d01b0ae9d43e34511c43c77f8a080bf0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43454477
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords--;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    +        ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
    +
    +        long released = 0L;
    +        while (dataPages.size() > 0) {
    +          MemoryBlock block = dataPages.getLast();
    +          // The currentPage is used, cannot be released
    +          if (block == currentPage) {
    +            break;
    +          }
    +
    +          Object base = block.getBaseObject();
    +          long offset = block.getBaseOffset();
    +          int numRecords = Platform.getInt(base, offset);
    +          offset += 4;
    +          final UnsafeSorterSpillWriter writer =
    +            new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords);
    +          while (numRecords-- > 0) {
    +            int length = Platform.getInt(base, offset);
    +            writer.write(base, offset + 4, length, 0);
    +            offset += 4 + length;
    +          }
    +          writer.close();
    +          spillWriters.add(writer);
    +          if (TaskContext.get() != null) {
    +            TaskContext.get().addOnCompleteCallback(new AbstractFunction0<BoxedUnit>() {
    +              @Override
    +              public BoxedUnit apply() {
    +                File file = writer.getFile();
    +                if (file != null && file.exists()) {
    +                  if (!file.delete()) {
    +                    logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +                  }
    +                }
    +                return null;
    +              }
    +            });
    +          }
    +
    +          dataPages.removeLast();
    +          freePage(block);
    +          released += block.size();
    +
    +          if (released > numBytes) {
    --- End diff --
    
    `>=` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#discussion_r43187750
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,95 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      long old = 0L;
    +      if (consumers.containsKey(consumer)) {
    +        old = consumers.get(consumer);
    +      }
    +      consumers.put(consumer, got + old);
    +
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    -    memoryManager.releaseExecutionMemory(size, taskAttemptId);
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    --- End diff --
    
    Add an assert to make sure `size >= 0`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152089767
  
    **[Test build #44572 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44572/consoleFull)** for PR 9241 at commit [`a3e01d0`](https://github.com/apache/spark/commit/a3e01d0c9222bee4bc8eb48e2542a4344e72aa33).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/9241#issuecomment-151939041
  
    @JoshRosen This PR is already huge, i will stop to add new things (but will fix any problems), please do another round review, thanks!
    
    For thread-safety, here what I'm got:
    
    1) Without calling spill(), the operators should only be used by single thread, no safety problems.
    
    2) spill() could be triggered in two cases, triggered by itself, or by other operators. we can check `trigger ==  this` in `spill()`, so it's still in the same thread, so safety problems.
    
    3) if it's triggered by other operators (right now cache will not trigger spill()), we only spill the data into disk when it's in scanning stage (building is finished), so the in-memory sorter or memory pages are read-only, we only need to synchronize the iterator and change it.
    
    4) During scanning, the iterator will only use one record in one page, we can't free this page, because the downstream is currently using it (used by UnsafeRow or other objects). In BytesToBytesMap, we just skip the current page, and dump all others into disk. In UnsafeExternalSorter, we keep the page that is used by current record (having the same baseObject), free it when loading the next record. In ShuffleExternalSorter, the spill() will not trigger during scanning.
    
    5) In order to avoid deadlock, we didn't call acquireMemory during spill (so we reused the pointer array in InMemorySorter). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43311051
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,97 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    assert(size >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got, consumer);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    --- End diff --
    
    Maybe add an assert got == 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150727452
  
    **[Test build #44275 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44275/consoleFull)** for PR 9241 at commit [`86e47ca`](https://github.com/apache/spark/commit/86e47ca4a459fce086fb9a499189c19fb4044111).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953601
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java ---
    @@ -219,6 +216,18 @@ public void spill() throws IOException {
         initializeForWriting();
       }
     
    +  public long release(long numBytes) throws IOException {
    --- End diff --
    
    `@Override`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152371889
  
    **[Test build #44652 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44652/consoleFull)** for PR 9241 at commit [`4ee1f42`](https://github.com/apache/spark/commit/4ee1f42b08b46469d8631a56b01898d251ebf3e7).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43452468
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    +      currentPage = allocatePage(required);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
        * Write a record to the shuffle sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      int partitionId) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId)
    +    throws IOException {
     
    -    if (numRecordsInsertedSinceLastSpill > numElementsForSpillThreshold) {
    +    // for tests
    +    assert(inMemSorter != null);
    +    if (inMemSorter.numRecords() > numElementsForSpillThreshold) {
           spill();
         }
     
         growPointerArrayIfNecessary();
    -    // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    --- End diff --
    
    https://issues.apache.org/jira/browse/SPARK-11411


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151574812
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43459046
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java ---
    @@ -83,37 +83,43 @@ public UnsafeInMemorySorter(
           final PrefixComparator prefixComparator,
           int initialSize) {
         assert (initialSize > 0);
    -    this.pointerArray = new long[initialSize * 2];
    +    this.array = new long[initialSize * 2];
         this.memoryManager = memoryManager;
         this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE);
         this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager);
       }
     
    +  public void reset() {
    +    pos = 0;
    +  }
    +
       /**
        * @return the number of records that have been inserted into this sorter.
        */
       public int numRecords() {
    -    return pointerArrayInsertPosition / 2;
    +    return pos / 2;
       }
     
    -  public long getMemoryUsage() {
    -    return pointerArray.length * 8L;
    +  private int newLength() {
    +    return array.length < Integer.MAX_VALUE / 2 ? (array.length * 2) : Integer.MAX_VALUE;
    +  }
    +
    +  public long getMemoryToExpand() {
    +    return (long) (newLength() - array.length) * 8L;
    --- End diff --
    
    Potential corner-case: this is the amount of _additional_ memory that will be used after the expansion has been performed successfully. _During_ the expansion, though, we hold onto both the old array and the new one. Do we want to account for this detail in our memory accounting?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151570108
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150684162
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43461668
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -647,16 +707,24 @@ public boolean putNewKey(
        * Acquire a new page from the memory manager.
        * @return whether there is enough space to allocate the new page.
        */
    -  private boolean acquireNewPage() {
    -    MemoryBlock newPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -    if (newPage == null) {
    -      logger.debug("Failed to acquire {} bytes of memory", pageSizeBytes);
    +  private boolean acquireNewPage(long required) {
    +    try {
    +      currentPage = allocatePage(required);
    +      dataPages.add(currentPage);
    +      Platform.putInt(currentPage.getBaseObject(), currentPage.getBaseOffset(), 0);
    +      pageCursor = 4;
    +      return true;
    +    } catch (OutOfMemoryError e) {
    --- End diff --
    
    If we want to catch OOME here, I think that we should do it at a much smaller scope (in the assignment to `currentPage` but not for adding to `dataPages` or modifying the page cursor. Given the risks of catching OOME that I mentioned above, the scope of the catch should be as narrow 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 pull request: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953237
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java ---
    @@ -312,107 +320,46 @@ private void growPointerArrayIfNecessary() throws IOException {
        * memory from the {@link ShuffleMemoryManager} and spill if the requested memory can not be
        * obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    assert (requiredSpace <= pageSizeBytes);
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    +  private void acquireNewPageIfNecessary(int required) throws IOException {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size()) {
           // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
           // without using the free space at the end of the current page. We should also do this for
           // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        acquireNewPage();
    +      long granted = shuffleMemoryManager.tryToAcquire(Math.max(required, pageSizeBytes), this);
    +      if (granted < required) {
    +        shuffleMemoryManager.release(granted, this);
    +        throw new IOException("Unable to acquire " + required + " bytes of memory");
           }
    +      currentPage = taskMemoryManager.allocatePage(granted);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
    -   * Acquire a new page from the {@link ShuffleMemoryManager}.
    -   *
    -   * If there is not enough space to allocate the new page, spill all existing ones
    -   * and try again. If there is still not enough space, report error to the caller.
    -   */
    -  private void acquireNewPage() throws IOException {
    -    final long memoryAcquired = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
    -    if (memoryAcquired < pageSizeBytes) {
    -      shuffleMemoryManager.release(memoryAcquired);
    -      spill();
    -      final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
    -      if (memoryAcquiredAfterSpilling != pageSizeBytes) {
    -        shuffleMemoryManager.release(memoryAcquiredAfterSpilling);
    -        throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -      }
    -    }
    -    currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -    currentPagePosition = currentPage.getBaseOffset();
    -    freeSpaceInCurrentPage = pageSizeBytes;
    -    allocatedPages.add(currentPage);
    -  }
    -
    -  /**
        * Write a record to the sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      long prefix) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, long prefix)
    +    throws IOException {
     
         growPointerArrayIfNecessary();
         // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    -      long overflowPageSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(totalSpaceRequired);
    --- End diff --
    
    Do we still need this `roundNumberOfBytesToNearestWord` method now that the memory allocators do them rounding themselves?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152352991
  
    **[Test build #44631 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44631/consoleFull)** for PR 9241 at commit [`c044afe`](https://github.com/apache/spark/commit/c044afe62a36d98934d4e9f36665e2a4837783c7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152111684
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152347313
  
    **[Test build #44644 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44644/consoleFull)** for PR 9241 at commit [`afc8c7c`](https://github.com/apache/spark/commit/afc8c7c9b0e92f9db6a5f72c14d8484a87311a51).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458390
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords--;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    +        ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
    +
    +        long released = 0L;
    +        while (dataPages.size() > 0) {
    +          MemoryBlock block = dataPages.getLast();
    +          // The currentPage is used, cannot be released
    +          if (block == currentPage) {
    +            break;
    +          }
    +
    +          Object base = block.getBaseObject();
    +          long offset = block.getBaseOffset();
    +          int numRecords = Platform.getInt(base, offset);
    +          offset += 4;
    +          final UnsafeSorterSpillWriter writer =
    +            new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords);
    +          while (numRecords-- > 0) {
    +            int length = Platform.getInt(base, offset);
    +            writer.write(base, offset + 4, length, 0);
    +            offset += 4 + length;
    +          }
    +          writer.close();
    +          spillWriters.add(writer);
    --- End diff --
    
    yes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150520127
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152373511
  
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150651722
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953261
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala ---
    @@ -27,6 +29,11 @@ import org.apache.spark.memory.{StaticMemoryManager, MemoryManager}
     import org.apache.spark.storage.{BlockId, BlockStatus}
     import org.apache.spark.unsafe.array.ByteArrayMethods
     
    +abstract class MemoryConsumer {
    +  @throws[IOException]
    +  def release(numBytes: Long): Long
    --- End diff --
    
    We should add Scaladoc here to clarify that this is asking the consumer to release memory but that the consumer might release more or less memory than we asked for (or none at all!). Should we give this method a different name that suggests this, such as `askToRelease()`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43454416
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    +            File file = spillWriters.removeFirst().getFile();
    +            if (file != null && file.exists()) {
    +              if (!file.delete()) {
    +                logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +              }
    +            }
    +          }
    +          try {
    +            reader = spillWriters.getFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords--;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    +        ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
    +
    +        long released = 0L;
    +        while (dataPages.size() > 0) {
    +          MemoryBlock block = dataPages.getLast();
    +          // The currentPage is used, cannot be released
    +          if (block == currentPage) {
    +            break;
    +          }
    +
    +          Object base = block.getBaseObject();
    +          long offset = block.getBaseOffset();
    +          int numRecords = Platform.getInt(base, offset);
    +          offset += 4;
    +          final UnsafeSorterSpillWriter writer =
    +            new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords);
    +          while (numRecords-- > 0) {
    +            int length = Platform.getInt(base, offset);
    +            writer.write(base, offset + 4, length, 0);
    +            offset += 4 + length;
    +          }
    +          writer.close();
    +          spillWriters.add(writer);
    +          if (TaskContext.get() != null) {
    +            TaskContext.get().addOnCompleteCallback(new AbstractFunction0<BoxedUnit>() {
    +              @Override
    +              public BoxedUnit apply() {
    +                File file = writer.getFile();
    +                if (file != null && file.exists()) {
    +                  if (!file.delete()) {
    +                    logger.error("Was unable to delete spill file {}", file.getAbsolutePath());
    +                  }
    +                }
    +                return null;
    +              }
    +            });
    +          }
    +
    +          dataPages.removeLast();
    +          freePage(block);
    +          released += block.size();
    --- End diff --
    
    You might want to record the block's size before calling `release()` above, just to be safe.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458814
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java ---
    @@ -312,107 +320,46 @@ private void growPointerArrayIfNecessary() throws IOException {
        * memory from the {@link ShuffleMemoryManager} and spill if the requested memory can not be
        * obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    assert (requiredSpace <= pageSizeBytes);
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    +  private void acquireNewPageIfNecessary(int required) throws IOException {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size()) {
           // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
           // without using the free space at the end of the current page. We should also do this for
           // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        acquireNewPage();
    +      long granted = shuffleMemoryManager.tryToAcquire(Math.max(required, pageSizeBytes), this);
    +      if (granted < required) {
    +        shuffleMemoryManager.release(granted, this);
    +        throw new IOException("Unable to acquire " + required + " bytes of memory");
           }
    +      currentPage = taskMemoryManager.allocatePage(granted);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
    -   * Acquire a new page from the {@link ShuffleMemoryManager}.
    -   *
    -   * If there is not enough space to allocate the new page, spill all existing ones
    -   * and try again. If there is still not enough space, report error to the caller.
    -   */
    -  private void acquireNewPage() throws IOException {
    -    final long memoryAcquired = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
    -    if (memoryAcquired < pageSizeBytes) {
    -      shuffleMemoryManager.release(memoryAcquired);
    -      spill();
    -      final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
    -      if (memoryAcquiredAfterSpilling != pageSizeBytes) {
    -        shuffleMemoryManager.release(memoryAcquiredAfterSpilling);
    -        throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -      }
    -    }
    -    currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -    currentPagePosition = currentPage.getBaseOffset();
    -    freeSpaceInCurrentPage = pageSizeBytes;
    -    allocatedPages.add(currentPage);
    -  }
    -
    -  /**
        * Write a record to the sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      long prefix) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, long prefix)
    +    throws IOException {
     
         growPointerArrayIfNecessary();
         // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    -      long overflowPageSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(totalSpaceRequired);
    --- End diff --
    
    It's still used by others (UnsafeWriter).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152126838
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44579/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/9241#issuecomment-152442966
  
    @JoshRosen I'm merging this into master, other comments will be addressed by followup 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43318196
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,27 +106,95 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long size, MemoryConsumer consumer) throws IOException {
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +
    +      // call spill() on itself to release some memory
    +      if (got < size && consumer != null) {
    +        consumer.spill(size - got);
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      if (got < size) {
    +        long needed = size - got;
    +        // call spill() on other consumers to release memory
    +        for (MemoryConsumer c: consumers.keySet()) {
    +          if (c != null && c != consumer) {
    +            needed -= c.spill(size - got);
    +            if (needed < 0) {
    +              break;
    +            }
    +          }
    +        }
    +        got += memoryManager.acquireExecutionMemory(size - got, taskAttemptId);
    +      }
    +
    +      long old = 0L;
    +      if (consumers.containsKey(consumer)) {
    +        old = consumers.get(consumer);
    +      }
    +      consumers.put(consumer, got + old);
    +
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    -    memoryManager.releaseExecutionMemory(size, taskAttemptId);
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    --- End diff --
    
    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 pull request: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43452800
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -219,61 +232,156 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class MapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
    +    private UnsafeSorterSpillReader reader = null;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private MapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    +      if (destructive) {
    +        destructiveIterator = this;
           }
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          freePage(currentPage);
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          if (reader != null) {
    +            // remove the spill file from disk
    --- End diff --
    
    Actually there is only one spilled file here, this is not needed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152111566
  
    **[Test build #44569 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44569/consoleFull)** for PR 9241 at commit [`0971d27`](https://github.com/apache/spark/commit/0971d274de4b642a43b2898c8f535f9721914dcb).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151755298
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44491/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152436958
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44656/
    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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150727333
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43450812
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    +      currentPage = allocatePage(required);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
        * Write a record to the shuffle sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      int partitionId) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId)
    +    throws IOException {
     
    -    if (numRecordsInsertedSinceLastSpill > numElementsForSpillThreshold) {
    +    // for tests
    +    assert(inMemSorter != null);
    +    if (inMemSorter.numRecords() > numElementsForSpillThreshold) {
           spill();
         }
     
         growPointerArrayIfNecessary();
    -    // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    --- End diff --
    
    In this old logic, a single big record which triggered an overflow page would not cause `currentPage` to be updated; I think that we should preserve the same behavior in 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 pull request: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458334
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    +      currentPage = allocatePage(required);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
        * Write a record to the shuffle sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      int partitionId) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId)
    +    throws IOException {
     
    -    if (numRecordsInsertedSinceLastSpill > numElementsForSpillThreshold) {
    +    // for tests
    +    assert(inMemSorter != null);
    +    if (inMemSorter.numRecords() > numElementsForSpillThreshold) {
           spill();
         }
     
         growPointerArrayIfNecessary();
    -    // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    -      long overflowPageSize = ByteArrayMethods.roundNumberOfBytesToNearestWord(totalSpaceRequired);
    -      // The record is larger than the page size, so allocate a special overflow page just to hold
    -      // that record.
    -      MemoryBlock overflowPage = taskMemoryManager.allocatePage(overflowPageSize);
    -      if (overflowPage == null) {
    -        spill();
    -        overflowPage = taskMemoryManager.allocatePage(overflowPageSize);
    -        if (overflowPage == null) {
    -          throw new IOException("Unable to acquire " + overflowPageSize + " bytes of memory");
    -        }
    -      }
    -      allocatedPages.add(overflowPage);
    -      dataPage = overflowPage;
    -      dataPagePosition = overflowPage.getBaseOffset();
    -    } else {
    -      // The record is small enough to fit in a regular data page, but the current page might not
    -      // have enough space to hold it (or no pages have been allocated yet).
    -      acquireNewPageIfNecessary(totalSpaceRequired);
    -      dataPage = currentPage;
    -      dataPagePosition = currentPagePosition;
    -      // Update bookkeeping information
    -      freeSpaceInCurrentPage -= totalSpaceRequired;
    -      currentPagePosition += totalSpaceRequired;
    -    }
    -    final Object dataPageBaseObject = dataPage.getBaseObject();
    -
    -    final long recordAddress =
    -      taskMemoryManager.encodePageNumberAndOffset(dataPage, dataPagePosition);
    -    Platform.putInt(dataPageBaseObject, dataPagePosition, lengthInBytes);
    -    dataPagePosition += 4;
    -    Platform.copyMemory(
    -      recordBaseObject, recordBaseOffset, dataPageBaseObject, dataPagePosition, lengthInBytes);
    -    assert(inMemSorter != null);
    +    final int required = length + 4;
    --- End diff --
    
    Added.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43443838
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // consumers could be modified by spill(), so we should have a copy here.
    +        MemoryConsumer[] cs = new MemoryConsumer[consumers.size()];
    +        consumers.keySet().toArray(cs);
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: cs) {
    +          if (c != null && c != consumer) {
    +            try {
    +              long released = c.spill(required - got, consumer);
    +              if (released > 0) {
    +                logger.info("Task {} released {} from {} for {}", taskAttemptId,
    +                  Utils.bytesToString(released), c, consumer);
    +                got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +                if (got >= required) {
    +                  break;
    +                }
    +              }
    +            } catch (IOException e) {
    +              logger.error("error while calling spill() on " + c, e);
    +              throw new OutOfMemoryError("error while calling spill() on " + c + " : "
    +                + e.getMessage());
    +            }
    +          }
    +        }
    +      }
    +
    +      // call spill() on itself
    +      if (got < required && consumer != null) {
    +        try {
    +          long released = consumer.spill(required - got, consumer);
    +          if (released > 0) {
    +            logger.info("Task {} released {} from itself ({})", taskAttemptId,
    +              Utils.bytesToString(released), consumer);
    +            got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +          }
    +        } catch (IOException e) {
    +          logger.error("error while calling spill() on " + consumer, e);
    +          throw new OutOfMemoryError("error while calling spill() on " + consumer + " : "
    +            + e.getMessage());
    +        }
    +      }
    +
    +      // Update the accounting, even consumer is null
    +      if (got > 0) {
    +        long old = 0L;
    +        if (consumers.containsKey(consumer)) {
    +          old = consumers.get(consumer);
    +        }
    +        consumers.put(consumer, got + old);
    +      }
    +
    +      logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer);
    +      return got;
    +    }
       }
     
       /**
    -   * Release N bytes of execution memory.
    +   * Release N bytes of execution memory for a MemoryConsumer.
        */
    -  public void releaseExecutionMemory(long size) {
    +  public void releaseExecutionMemory(long size, MemoryConsumer consumer) {
    +    assert(size >= 0);
    +    if (size == 0) {
    +      return;
    +    }
    +    synchronized (this) {
    +      if (consumers.containsKey(consumer)) {
    +        long old = consumers.get(consumer);
    +        if (old > size) {
    +          consumers.put(consumer, old - size);
    +        } else {
    +          if (old < size) {
    +            String msg = "Release " + size + " bytes memory (more than acquired " + old + ") for "
    +              + consumer;
    +            logger.warn(msg);
    +            if (Utils.isTesting()) {
    +              Platform.throwException(new SparkException(msg));
    +            }
    +          }
    +          consumers.remove(consumer);
    +        }
    +      } else {
    +        String msg = "Release " + size + " bytes memory for non-existent " + consumer;
    +        logger.warn(msg);
    +        if (Utils.isTesting()) {
    +          Platform.throwException(new SparkException(msg));
    +        }
    +      }
    +    }
    +
    +    logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer);
         memoryManager.releaseExecutionMemory(size, taskAttemptId);
       }
     
    +  public void transferOwnership(long size, MemoryConsumer from, MemoryConsumer to) {
    --- End diff --
    
    Yes, will be removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151581339
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150520132
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44210/
    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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151725769
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43450295
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -353,15 +320,22 @@ private void growPointerArrayIfNecessary() throws IOException {
         assert(inMemSorter != null);
         if (!inMemSorter.hasSpaceForAnotherRecord()) {
           logger.debug("Attempting to expand sort pointer array");
    -      final long oldPointerArrayMemoryUsage = inMemSorter.getMemoryUsage();
    -      final long memoryToGrowPointerArray = oldPointerArrayMemoryUsage * 2;
    -      final long memoryAcquired = taskMemoryManager.acquireExecutionMemory(memoryToGrowPointerArray);
    -      if (memoryAcquired < memoryToGrowPointerArray) {
    -        taskMemoryManager.releaseExecutionMemory(memoryAcquired);
    -        spill();
    -      } else {
    -        inMemSorter.expandPointerArray();
    -        taskMemoryManager.releaseExecutionMemory(oldPointerArrayMemoryUsage);
    +      long used = inMemSorter.getMemoryUsage();
    +      long needed = inMemSorter.getMemoryToExpand();
    +      try {
    +        acquireMemory(used + needed);  // could trigger spilling
    +        if (inMemSorter.hasSpaceForAnotherRecord()) {
    +          releaseMemory(used + needed);
    +        } else {
    +          logger.debug("Expand sort pointer array");
    +          inMemSorter.expandPointerArray();
    +          releaseMemory(used);
    +        }
    +      } catch (OutOfMemoryError oom) {
    --- End diff --
    
    We probably shouldn't catch `OutOfMemoryError`, since this could lead to problems in case we catch an OOM thrown by the JVM and not by our own code. Instead, we might want to throw our own `SparkOutOfMemoryError` exception or something like that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43460397
  
    --- Diff: core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java ---
    @@ -83,37 +83,43 @@ public UnsafeInMemorySorter(
           final PrefixComparator prefixComparator,
           int initialSize) {
         assert (initialSize > 0);
    -    this.pointerArray = new long[initialSize * 2];
    +    this.array = new long[initialSize * 2];
         this.memoryManager = memoryManager;
         this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE);
         this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager);
       }
     
    +  public void reset() {
    +    pos = 0;
    +  }
    +
       /**
        * @return the number of records that have been inserted into this sorter.
        */
       public int numRecords() {
    -    return pointerArrayInsertPosition / 2;
    +    return pos / 2;
       }
     
    -  public long getMemoryUsage() {
    -    return pointerArray.length * 8L;
    +  private int newLength() {
    +    return array.length < Integer.MAX_VALUE / 2 ? (array.length * 2) : Integer.MAX_VALUE;
    +  }
    +
    +  public long getMemoryToExpand() {
    +    return (long) (newLength() - array.length) * 8L;
    --- End diff --
    
    Discussed offline; this is actually accounted for at the call site.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43458998
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -353,15 +320,22 @@ private void growPointerArrayIfNecessary() throws IOException {
         assert(inMemSorter != null);
         if (!inMemSorter.hasSpaceForAnotherRecord()) {
           logger.debug("Attempting to expand sort pointer array");
    -      final long oldPointerArrayMemoryUsage = inMemSorter.getMemoryUsage();
    -      final long memoryToGrowPointerArray = oldPointerArrayMemoryUsage * 2;
    -      final long memoryAcquired = taskMemoryManager.acquireExecutionMemory(memoryToGrowPointerArray);
    -      if (memoryAcquired < memoryToGrowPointerArray) {
    -        taskMemoryManager.releaseExecutionMemory(memoryAcquired);
    -        spill();
    -      } else {
    -        inMemSorter.expandPointerArray();
    -        taskMemoryManager.releaseExecutionMemory(oldPointerArrayMemoryUsage);
    +      long used = inMemSorter.getMemoryUsage();
    +      long needed = inMemSorter.getMemoryToExpand();
    +      try {
    +        acquireMemory(used + needed);  // could trigger spilling
    +        if (inMemSorter.hasSpaceForAnotherRecord()) {
    +          releaseMemory(used + needed);
    +        } else {
    +          logger.debug("Expand sort pointer array");
    +          inMemSorter.expandPointerArray();
    +          releaseMemory(used);
    +        }
    +      } catch (OutOfMemoryError oom) {
    --- End diff --
    
    We are allocate a huge object here, so it's basically recoverable. It's better to have them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43454924
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -444,6 +545,19 @@ private Location with(MemoryBlock page, long offsetInPage) {
         }
     
         /**
    +     +     * This is only used for spilling
    --- End diff --
    
    Indentation problem?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953355
  
    --- Diff: core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala ---
    @@ -152,6 +199,11 @@ class ShuffleMemoryManager protected (
         taskMemory.remove(taskAttemptId).foreach { numBytes =>
           memoryManager.releaseExecutionMemory(numBytes)
         }
    +    taskConsumers.remove(taskAttemptId).map { consumers =>
    +      consumers.foreach { case (consumer, numBytes) =>
    +        logWarning(s"Memory leak in task $taskAttemptId: $consumer $numBytes bytes")
    --- End diff --
    
    Same here; I think this should be an exception in unit tests and a warning in production.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151585757
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43449616
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // consumers could be modified by spill(), so we should have a copy here.
    +        MemoryConsumer[] cs = new MemoryConsumer[consumers.size()];
    +        consumers.keySet().toArray(cs);
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: cs) {
    +          if (c != null && c != consumer) {
    --- End diff --
    
    There could be some cases the memory is acquire not by a consumer, it will be null.


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

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


[GitHub] spark pull request: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150703411
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44263/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152360297
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150653550
  
    **[Test build #44242 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44242/consoleFull)** for PR 9241 at commit [`0c77c94`](https://github.com/apache/spark/commit/0c77c94c9d50ad8935a5b3009a25e21966193987).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#issuecomment-150702314
  
    **[Test build #44265 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44265/consoleFull)** for PR 9241 at commit [`5c198cf`](https://github.com/apache/spark/commit/5c198cf329748a0af9f108d182615616334a6c6a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152034329
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44542/
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43310368
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public abstract class MemoryConsumer {
    --- End diff --
    
    How about naming this class SpillableMemoryConsumer ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43457018
  
    --- Diff: core/src/main/java/org/apache/spark/memory/MemoryConsumer.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.memory;
    +
    +
    +import java.io.IOException;
    +
    +import org.apache.spark.unsafe.memory.MemoryBlock;
    +
    +
    +/**
    + * An memory consumer of TaskMemoryManager, which support spilling.
    + */
    +public abstract class MemoryConsumer {
    +
    +  private final TaskMemoryManager taskMemoryManager;
    +  private final long pageSize;
    +  private long used;
    +
    +  protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize) {
    +    this.taskMemoryManager = taskMemoryManager;
    +    if (pageSize == 0) {
    --- End diff --
    
    Is this necessary since we have another constructor?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43442273
  
    --- Diff: core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java ---
    @@ -101,29 +107,161 @@
       private final boolean inHeap;
     
       /**
    +   * The size of memory granted to each consumer.
    +   */
    +  private final HashMap<MemoryConsumer, Long> consumers;
    +
    +  /**
        * Construct a new TaskMemoryManager.
        */
       public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) {
         this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap();
         this.memoryManager = memoryManager;
         this.taskAttemptId = taskAttemptId;
    +    this.consumers = new HashMap<>();
       }
     
       /**
    -   * Acquire N bytes of memory for execution, evicting cached blocks if necessary.
    +   * Acquire N bytes of memory for a consumer. If there is no enough memory, it will call
    +   * spill() of consumers to release more memory.
    +   *
        * @return number of bytes successfully granted (<= N).
        */
    -  public long acquireExecutionMemory(long size) {
    -    return memoryManager.acquireExecutionMemory(size, taskAttemptId);
    +  public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
    +    assert(required >= 0);
    +    synchronized (this) {
    +      long got = memoryManager.acquireExecutionMemory(required, taskAttemptId);
    +
    +      // try to release memory from other consumers first, then we can reduce the frequency of
    +      // spilling, avoid to have too many spilled files.
    +      if (got < required) {
    +        // consumers could be modified by spill(), so we should have a copy here.
    +        MemoryConsumer[] cs = new MemoryConsumer[consumers.size()];
    +        consumers.keySet().toArray(cs);
    +        // Call spill() on other consumers to release memory
    +        for (MemoryConsumer c: cs) {
    +          if (c != null && c != consumer) {
    +            try {
    +              long released = c.spill(required - got, consumer);
    +              if (released > 0) {
    +                logger.info("Task {} released {} from {} for {}", taskAttemptId,
    +                  Utils.bytesToString(released), c, consumer);
    +                got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId);
    +                if (got >= required) {
    +                  break;
    +                }
    +              }
    +            } catch (IOException e) {
    +              logger.error("error while calling spill() on " + c, e);
    +              throw new OutOfMemoryError("error while calling spill() on " + c + " : "
    +                + e.getMessage());
    --- End diff --
    
    Instead of using string concatenation to pass the IOException's message, why not use regular exception chaining here? Does `OutOfMemoryError` not support that?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152399801
  
    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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152360735
  
    **[Test build #44647 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44647/consoleFull)** for PR 9241 at commit [`cda4b2a`](https://github.com/apache/spark/commit/cda4b2afa45acef529eb24afa7ec0c5370de237f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#issuecomment-152085173
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151773563
  
    **[Test build #44504 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44504/consoleFull)** for PR 9241 at commit [`8470fc9`](https://github.com/apache/spark/commit/8470fc9ddd37c525cf648c00e019d968410fa66f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public abstract class MemoryConsumer `\n  * `final class ShuffleExternalSorter extends MemoryConsumer `\n  * `public final class BytesToBytesMap extends MemoryConsumer `\n  * `  public final class MapIterator implements Iterator<Location> `\n  * `public final class UnsafeExternalSorter extends MemoryConsumer `\n  * `  class SpillableIterator extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillReader extends UnsafeSorterIterator `\n  * `public final class UnsafeSorterSpillWriter `\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SQL] [WIP] Cooperative memory m...

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

    https://github.com/apache/spark/pull/9241#discussion_r42953420
  
    --- Diff: core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java ---
    @@ -227,62 +238,147 @@ public BytesToBytesMap(
        */
       public int numElements() { return numElements; }
     
    -  public static final class BytesToBytesMapIterator implements Iterator<Location> {
    +  public final class BytesToBytesMapIterator implements Iterator<Location> {
     
    -    private final int numRecords;
    -    private final Iterator<MemoryBlock> dataPagesIterator;
    +    private int numRecords;
         private final Location loc;
     
         private MemoryBlock currentPage = null;
    -    private int currentRecordNumber = 0;
    +    private int recordsInPage = 0;
         private Object pageBaseObject;
         private long offsetInPage;
     
         // If this iterator destructive or not. When it is true, it frees each page as it moves onto
         // next one.
         private boolean destructive = false;
    -    private BytesToBytesMap bmap;
     
    -    private BytesToBytesMapIterator(
    -        int numRecords, Iterator<MemoryBlock> dataPagesIterator, Location loc,
    -        boolean destructive, BytesToBytesMap bmap) {
    +    private LinkedList<UnsafeSorterSpillWriter> spillWriters =
    +      new LinkedList<UnsafeSorterSpillWriter>();
    +    private UnsafeSorterSpillReader reader = null;
    +
    +    private BytesToBytesMapIterator(int numRecords, Location loc, boolean destructive) {
           this.numRecords = numRecords;
    -      this.dataPagesIterator = dataPagesIterator;
           this.loc = loc;
           this.destructive = destructive;
    -      this.bmap = bmap;
    -      if (dataPagesIterator.hasNext()) {
    -        advanceToNextPage();
    -      }
    +      destructiveIterator = this;
         }
     
         private void advanceToNextPage() {
    -      if (destructive && currentPage != null) {
    -        dataPagesIterator.remove();
    -        this.bmap.taskMemoryManager.freePage(currentPage);
    -        this.bmap.shuffleMemoryManager.release(currentPage.size());
    +      synchronized (this) {
    +        int nextIdx = dataPages.indexOf(currentPage) + 1;
    +        if (destructive && currentPage != null) {
    +          dataPages.remove(currentPage);
    +          taskMemoryManager.freePage(currentPage);
    +          shuffleMemoryManager.release(currentPage.size());
    +          nextIdx --;
    +        }
    +        if (dataPages.size() > nextIdx) {
    +          currentPage = dataPages.get(nextIdx);
    +          pageBaseObject = currentPage.getBaseObject();
    +          offsetInPage = currentPage.getBaseOffset();
    +          recordsInPage = Platform.getInt(pageBaseObject, offsetInPage);
    +          offsetInPage += 4;
    +        } else {
    +          currentPage = null;
    +          try {
    +            reader = spillWriters.removeFirst().getReader(blockManager);
    +            recordsInPage = -1;
    +          } catch (IOException e) {
    +            // Scala iterator does not handle exception
    +            Platform.throwException(e);
    +          }
    +        }
           }
    -      currentPage = dataPagesIterator.next();
    -      pageBaseObject = currentPage.getBaseObject();
    -      offsetInPage = currentPage.getBaseOffset();
         }
     
         @Override
         public boolean hasNext() {
    -      return currentRecordNumber != numRecords;
    +      return numRecords > 0;
         }
     
         @Override
         public Location next() {
    -      int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    -      if (totalLength == END_OF_PAGE_MARKER) {
    +      if (recordsInPage == 0) {
             advanceToNextPage();
    -        totalLength = Platform.getInt(pageBaseObject, offsetInPage);
           }
    -      loc.with(currentPage, offsetInPage);
    -      offsetInPage += 4 + totalLength;
    -      currentRecordNumber++;
    -      return loc;
    +      numRecords --;
    +      if (currentPage != null) {
    +        int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
    +        loc.with(currentPage, offsetInPage);
    +        offsetInPage += 4 + totalLength;
    +        recordsInPage --;
    +        return loc;
    +      } else {
    +        assert(reader != null);
    +        if (!reader.hasNext()) {
    +          advanceToNextPage();
    +        }
    +        try {
    +          reader.loadNext();
    +        } catch (IOException e) {
    +          // Scala iterator does not handle exception
    +          Platform.throwException(e);
    +        }
    +        loc.with(reader.getBaseObject(), reader.getBaseOffset(), reader.getRecordLength());
    +        return loc;
    +      }
    +    }
    +
    +    public long spill(long numBytes) throws IOException {
    +      synchronized (this) {
    +        if (!destructive || dataPages.size() == 1) {
    +          return 0L;
    +        }
    +
    +        // TODO: use existing ShuffleWriteMetrics
    --- End diff --
    
    Let's chat about this later; I'm not sure whether or not the existing spillable collections have consistent semantics in terms accounting for spills via shuffle write metrics. I spent a decent amount of time investigating this earlier in the year, so I'll see if I can find my notes from then.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPA...

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

    https://github.com/apache/spark/pull/9241#discussion_r43451491
  
    --- Diff: core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java ---
    @@ -370,96 +344,45 @@ private void growPointerArrayIfNecessary() throws IOException {
        * Allocates more memory in order to insert an additional record. This will request additional
        * memory from the memory manager and spill if the requested memory can not be obtained.
        *
    -   * @param requiredSpace the required space in the data page, in bytes, including space for storing
    +   * @param required the required space in the data page, in bytes, including space for storing
        *                      the record size. This must be less than or equal to the page size (records
        *                      that exceed the page size are handled via a different code path which uses
        *                      special overflow pages).
        */
    -  private void acquireNewPageIfNecessary(int requiredSpace) throws IOException {
    -    growPointerArrayIfNecessary();
    -    if (requiredSpace > freeSpaceInCurrentPage) {
    -      logger.trace("Required space {} is less than free space in current page ({})", requiredSpace,
    -        freeSpaceInCurrentPage);
    -      // TODO: we should track metrics on the amount of space wasted when we roll over to a new page
    -      // without using the free space at the end of the current page. We should also do this for
    -      // BytesToBytesMap.
    -      if (requiredSpace > pageSizeBytes) {
    -        throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
    -          pageSizeBytes + ")");
    -      } else {
    -        currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -        if (currentPage == null) {
    -          spill();
    -          currentPage = taskMemoryManager.allocatePage(pageSizeBytes);
    -          if (currentPage == null) {
    -            throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
    -          }
    -        }
    -        currentPagePosition = currentPage.getBaseOffset();
    -        freeSpaceInCurrentPage = pageSizeBytes;
    -        allocatedPages.add(currentPage);
    -      }
    +  private void acquireNewPageIfNecessary(int required) {
    +    if (currentPage == null ||
    +      pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) {
    +      // TODO: try to find space in previous pages
    +      currentPage = allocatePage(required);
    +      pageCursor = currentPage.getBaseOffset();
    +      allocatedPages.add(currentPage);
         }
       }
     
       /**
        * Write a record to the shuffle sorter.
        */
    -  public void insertRecord(
    -      Object recordBaseObject,
    -      long recordBaseOffset,
    -      int lengthInBytes,
    -      int partitionId) throws IOException {
    +  public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId)
    +    throws IOException {
     
    -    if (numRecordsInsertedSinceLastSpill > numElementsForSpillThreshold) {
    +    // for tests
    +    assert(inMemSorter != null);
    +    if (inMemSorter.numRecords() > numElementsForSpillThreshold) {
           spill();
         }
     
         growPointerArrayIfNecessary();
    -    // Need 4 bytes to store the record length.
    -    final int totalSpaceRequired = lengthInBytes + 4;
    -
    -    // --- Figure out where to insert the new record ----------------------------------------------
    -
    -    final MemoryBlock dataPage;
    -    long dataPagePosition;
    -    boolean useOverflowPage = totalSpaceRequired > pageSizeBytes;
    -    if (useOverflowPage) {
    --- End diff --
    
    Even with that, we still lose some space in each end of block. I think we can do that as 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: [SPARK-10342] [SPARK-10309] [SQL] [WIP] Cooper...

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

    https://github.com/apache/spark/pull/9241#issuecomment-151630052
  
    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