You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by da...@apache.org on 2015/10/30 07:39:10 UTC
[2/3] spark git commit: [SPARK-10342] [SPARK-10309] [SPARK-10474]
[SPARK-10929] [SQL] Cooperative memory management
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index e317ea3..49a5a4b 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -17,39 +17,34 @@
package org.apache.spark.util.collection.unsafe.sort;
+import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.LinkedList;
-import javax.annotation.Nullable;
-
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
-
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.spark.TaskContext;
import org.apache.spark.executor.ShuffleWriteMetrics;
+import org.apache.spark.memory.MemoryConsumer;
+import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.storage.BlockManager;
-import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.memory.MemoryBlock;
-import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.util.TaskCompletionListener;
import org.apache.spark.util.Utils;
/**
* External sorter based on {@link UnsafeInMemorySorter}.
*/
-public final class UnsafeExternalSorter {
+public final class UnsafeExternalSorter extends MemoryConsumer {
private final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class);
- private final long pageSizeBytes;
private final PrefixComparator prefixComparator;
private final RecordComparator recordComparator;
- private final int initialSize;
private final TaskMemoryManager taskMemoryManager;
private final BlockManager blockManager;
private final TaskContext taskContext;
@@ -69,14 +64,12 @@ public final class UnsafeExternalSorter {
private final LinkedList<UnsafeSorterSpillWriter> spillWriters = new LinkedList<>();
// These variables are reset after spilling:
- @Nullable private UnsafeInMemorySorter inMemSorter;
- // Whether the in-mem sorter is created internally, or passed in from outside.
- // If it is passed in from outside, we shouldn't release the in-mem sorter's memory.
- private boolean isInMemSorterExternal = false;
+ @Nullable private volatile UnsafeInMemorySorter inMemSorter;
+
private MemoryBlock currentPage = null;
- private long currentPagePosition = -1;
- private long freeSpaceInCurrentPage = 0;
+ private long pageCursor = -1;
private long peakMemoryUsedBytes = 0;
+ private volatile SpillableIterator readingIterator = null;
public static UnsafeExternalSorter createWithExistingInMemorySorter(
TaskMemoryManager taskMemoryManager,
@@ -86,7 +79,7 @@ public final class UnsafeExternalSorter {
PrefixComparator prefixComparator,
int initialSize,
long pageSizeBytes,
- UnsafeInMemorySorter inMemorySorter) throws IOException {
+ UnsafeInMemorySorter inMemorySorter) {
return new UnsafeExternalSorter(taskMemoryManager, blockManager,
taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, inMemorySorter);
}
@@ -98,7 +91,7 @@ public final class UnsafeExternalSorter {
RecordComparator recordComparator,
PrefixComparator prefixComparator,
int initialSize,
- long pageSizeBytes) throws IOException {
+ long pageSizeBytes) {
return new UnsafeExternalSorter(taskMemoryManager, blockManager,
taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, null);
}
@@ -111,60 +104,41 @@ public final class UnsafeExternalSorter {
PrefixComparator prefixComparator,
int initialSize,
long pageSizeBytes,
- @Nullable UnsafeInMemorySorter existingInMemorySorter) throws IOException {
+ @Nullable UnsafeInMemorySorter existingInMemorySorter) {
+ super(taskMemoryManager, pageSizeBytes);
this.taskMemoryManager = taskMemoryManager;
this.blockManager = blockManager;
this.taskContext = taskContext;
this.recordComparator = recordComparator;
this.prefixComparator = prefixComparator;
- this.initialSize = initialSize;
// Use getSizeAsKb (not bytes) to maintain backwards compatibility for units
// this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
this.fileBufferSizeBytes = 32 * 1024;
- this.pageSizeBytes = pageSizeBytes;
+ // TODO: metrics tracking + integration with shuffle write metrics
+ // need to connect the write metrics to task metrics so we count the spill IO somewhere.
this.writeMetrics = new ShuffleWriteMetrics();
if (existingInMemorySorter == null) {
- initializeForWriting();
- // Acquire a new page as soon as we construct the sorter to ensure that we have at
- // least one page to work with. Otherwise, other operators in the same task may starve
- // this sorter (SPARK-9709). We don't need to do this if we already have an existing sorter.
- acquireNewPage();
+ this.inMemSorter =
+ new UnsafeInMemorySorter(taskMemoryManager, recordComparator, prefixComparator, initialSize);
+ acquireMemory(inMemSorter.getMemoryUsage());
} else {
- this.isInMemSorterExternal = true;
this.inMemSorter = existingInMemorySorter;
+ // will acquire after free the map
}
+ this.peakMemoryUsedBytes = getMemoryUsage();
// Register a cleanup task with TaskContext to ensure that memory is guaranteed to be freed at
// the end of the task. This is necessary to avoid memory leaks in when the downstream operator
// does not fully consume the sorter's output (e.g. sort followed by limit).
- taskContext.addOnCompleteCallback(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- cleanupResources();
- return null;
+ taskContext.addTaskCompletionListener(
+ new TaskCompletionListener() {
+ @Override
+ public void onTaskCompletion(TaskContext context) {
+ cleanupResources();
+ }
}
- });
- }
-
- // TODO: metrics tracking + integration with shuffle write metrics
- // need to connect the write metrics to task metrics so we count the spill IO somewhere.
-
- /**
- * Allocates new sort data structures. Called when creating the sorter and after each spill.
- */
- private void initializeForWriting() throws IOException {
- // Note: Do not track memory for the pointer array for now because of SPARK-10474.
- // In more detail, in TungstenAggregate we only reserve a page, but when we fall back to
- // sort-based aggregation we try to acquire a page AND a pointer array, which inevitably
- // fails if all other memory is already occupied. It should be safe to not track the array
- // because its memory footprint is frequently much smaller than that of a page. This is a
- // temporary hack that we should address in 1.6.0.
- // TODO: track the pointer array memory!
- this.writeMetrics = new ShuffleWriteMetrics();
- this.inMemSorter =
- new UnsafeInMemorySorter(taskMemoryManager, recordComparator, prefixComparator, initialSize);
- this.isInMemSorterExternal = false;
+ );
}
/**
@@ -173,14 +147,27 @@ public final class UnsafeExternalSorter {
*/
@VisibleForTesting
public void closeCurrentPage() {
- freeSpaceInCurrentPage = 0;
+ if (currentPage != null) {
+ pageCursor = currentPage.getBaseOffset() + currentPage.size();
+ }
}
/**
* Sort and spill the current records in response to memory pressure.
*/
- public void spill() throws IOException {
- assert(inMemSorter != null);
+ @Override
+ public long spill(long size, MemoryConsumer trigger) throws IOException {
+ if (trigger != this) {
+ if (readingIterator != null) {
+ return readingIterator.spill();
+ }
+ return 0L;
+ }
+
+ if (inMemSorter == null || inMemSorter.numRecords() <= 0) {
+ return 0L;
+ }
+
logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)",
Thread.currentThread().getId(),
Utils.bytesToString(getMemoryUsage()),
@@ -202,6 +189,8 @@ public final class UnsafeExternalSorter {
spillWriter.write(baseObject, baseOffset, recordLength, sortedRecords.getKeyPrefix());
}
spillWriter.close();
+
+ inMemSorter.reset();
}
final long spillSize = freeMemory();
@@ -210,7 +199,7 @@ public final class UnsafeExternalSorter {
// written to disk. This also counts the space needed to store the sorter's pointer array.
taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
- initializeForWriting();
+ return spillSize;
}
/**
@@ -246,7 +235,7 @@ public final class UnsafeExternalSorter {
}
/**
- * Free this sorter's in-memory data structures, including its data pages and pointer array.
+ * Free this sorter's data pages.
*
* @return the number of bytes freed.
*/
@@ -254,14 +243,12 @@ public final class UnsafeExternalSorter {
updatePeakMemoryUsed();
long memoryFreed = 0;
for (MemoryBlock block : allocatedPages) {
- taskMemoryManager.freePage(block);
memoryFreed += block.size();
+ freePage(block);
}
- // TODO: track in-memory sorter memory usage (SPARK-10474)
allocatedPages.clear();
currentPage = null;
- currentPagePosition = -1;
- freeSpaceInCurrentPage = 0;
+ pageCursor = 0;
return memoryFreed;
}
@@ -283,8 +270,15 @@ public final class UnsafeExternalSorter {
* Frees this sorter's in-memory data structures and cleans up its spill files.
*/
public void cleanupResources() {
- deleteSpillFiles();
- freeMemory();
+ synchronized (this) {
+ deleteSpillFiles();
+ freeMemory();
+ if (inMemSorter != null) {
+ long used = inMemSorter.getMemoryUsage();
+ inMemSorter = null;
+ releaseMemory(used);
+ }
+ }
}
/**
@@ -295,8 +289,28 @@ public final class UnsafeExternalSorter {
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 = used + inMemSorter.getMemoryToExpand();
+ try {
+ acquireMemory(needed); // could trigger spilling
+ } catch (OutOfMemoryError e) {
+ // should have trigger spilling
+ assert(inMemSorter.hasSpaceForAnotherRecord());
+ return;
+ }
+ // check if spilling is triggered or not
+ if (inMemSorter.hasSpaceForAnotherRecord()) {
+ releaseMemory(needed);
+ } else {
+ try {
+ inMemSorter.expandPointerArray();
+ releaseMemory(used);
+ } catch (OutOfMemoryError oom) {
+ // Just in case that JVM had run out of memory
+ releaseMemory(needed);
+ spill();
+ }
+ }
}
}
@@ -304,101 +318,38 @@ public final class UnsafeExternalSorter {
* 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 {
- assert (requiredSpace <= pageSizeBytes);
- 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 {
- acquireNewPage();
- }
+ private void acquireNewPageIfNecessary(int required) {
+ if (currentPage == null ||
+ pageCursor + required > currentPage.getBaseOffset() + currentPage.size()) {
+ // TODO: try to find space on previous pages
+ currentPage = allocatePage(required);
+ pageCursor = currentPage.getBaseOffset();
+ allocatedPages.add(currentPage);
}
}
/**
- * Acquire a new page from the memory manager.
- *
- * 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 {
- 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);
- }
-
- /**
* 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);
- // 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();
-
- // --- Insert the record ----------------------------------------------------------------------
-
- final long recordAddress =
- taskMemoryManager.encodePageNumberAndOffset(dataPage, dataPagePosition);
- Platform.putInt(dataPageBaseObject, dataPagePosition, lengthInBytes);
- dataPagePosition += 4;
- Platform.copyMemory(
- recordBaseObject, recordBaseOffset, dataPageBaseObject, dataPagePosition, lengthInBytes);
+ final int required = length + 4;
+ acquireNewPageIfNecessary(required);
+
+ final Object base = currentPage.getBaseObject();
+ final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor);
+ Platform.putInt(base, pageCursor, length);
+ pageCursor += 4;
+ Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length);
+ pageCursor += length;
assert(inMemSorter != null);
inMemSorter.insertRecord(recordAddress, prefix);
}
@@ -411,59 +362,24 @@ public final class UnsafeExternalSorter {
*
* record length = key length + value length + 4
*/
- public void insertKVRecord(
- Object keyBaseObj, long keyOffset, int keyLen,
- Object valueBaseObj, long valueOffset, int valueLen, long prefix) throws IOException {
+ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen,
+ Object valueBase, long valueOffset, int valueLen, long prefix)
+ throws IOException {
growPointerArrayIfNecessary();
- final int totalSpaceRequired = keyLen + valueLen + 4 + 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();
-
- // --- Insert the record ----------------------------------------------------------------------
-
- final long recordAddress =
- taskMemoryManager.encodePageNumberAndOffset(dataPage, dataPagePosition);
- Platform.putInt(dataPageBaseObject, dataPagePosition, keyLen + valueLen + 4);
- dataPagePosition += 4;
-
- Platform.putInt(dataPageBaseObject, dataPagePosition, keyLen);
- dataPagePosition += 4;
-
- Platform.copyMemory(keyBaseObj, keyOffset, dataPageBaseObject, dataPagePosition, keyLen);
- dataPagePosition += keyLen;
-
- Platform.copyMemory(valueBaseObj, valueOffset, dataPageBaseObject, dataPagePosition, valueLen);
+ final int required = keyLen + valueLen + 4 + 4;
+ acquireNewPageIfNecessary(required);
+
+ final Object base = currentPage.getBaseObject();
+ final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor);
+ Platform.putInt(base, pageCursor, keyLen + valueLen + 4);
+ pageCursor += 4;
+ Platform.putInt(base, pageCursor, keyLen);
+ pageCursor += 4;
+ Platform.copyMemory(keyBase, keyOffset, base, pageCursor, keyLen);
+ pageCursor += keyLen;
+ Platform.copyMemory(valueBase, valueOffset, base, pageCursor, valueLen);
+ pageCursor += valueLen;
assert(inMemSorter != null);
inMemSorter.insertRecord(recordAddress, prefix);
@@ -475,10 +391,10 @@ public final class UnsafeExternalSorter {
*/
public UnsafeSorterIterator getSortedIterator() throws IOException {
assert(inMemSorter != null);
- final UnsafeInMemorySorter.SortedIterator inMemoryIterator = inMemSorter.getSortedIterator();
- int numIteratorsToMerge = spillWriters.size() + (inMemoryIterator.hasNext() ? 1 : 0);
+ readingIterator = new SpillableIterator(inMemSorter.getSortedIterator());
+ int numIteratorsToMerge = spillWriters.size() + (readingIterator.hasNext() ? 1 : 0);
if (spillWriters.isEmpty()) {
- return inMemoryIterator;
+ return readingIterator;
} else {
final UnsafeSorterSpillMerger spillMerger =
new UnsafeSorterSpillMerger(recordComparator, prefixComparator, numIteratorsToMerge);
@@ -486,9 +402,113 @@ public final class UnsafeExternalSorter {
spillMerger.addSpillIfNotEmpty(spillWriter.getReader(blockManager));
}
spillWriters.clear();
- spillMerger.addSpillIfNotEmpty(inMemoryIterator);
+ spillMerger.addSpillIfNotEmpty(readingIterator);
return spillMerger.getSortedIterator();
}
}
+
+ /**
+ * An UnsafeSorterIterator that support spilling.
+ */
+ class SpillableIterator extends UnsafeSorterIterator {
+ private UnsafeSorterIterator upstream;
+ private UnsafeSorterIterator nextUpstream = null;
+ private MemoryBlock lastPage = null;
+ private boolean loaded = false;
+ private int numRecords = 0;
+
+ public SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) {
+ this.upstream = inMemIterator;
+ this.numRecords = inMemIterator.numRecordsLeft();
+ }
+
+ public long spill() throws IOException {
+ synchronized (this) {
+ if (!(upstream instanceof UnsafeInMemorySorter.SortedIterator && nextUpstream == null
+ && numRecords > 0)) {
+ return 0L;
+ }
+
+ UnsafeInMemorySorter.SortedIterator inMemIterator =
+ ((UnsafeInMemorySorter.SortedIterator) upstream).clone();
+
+ final UnsafeSorterSpillWriter spillWriter =
+ new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, numRecords);
+ while (inMemIterator.hasNext()) {
+ inMemIterator.loadNext();
+ final Object baseObject = inMemIterator.getBaseObject();
+ final long baseOffset = inMemIterator.getBaseOffset();
+ final int recordLength = inMemIterator.getRecordLength();
+ spillWriter.write(baseObject, baseOffset, recordLength, inMemIterator.getKeyPrefix());
+ }
+ spillWriter.close();
+ spillWriters.add(spillWriter);
+ nextUpstream = spillWriter.getReader(blockManager);
+
+ long released = 0L;
+ synchronized (UnsafeExternalSorter.this) {
+ // release the pages except the one that is used
+ for (MemoryBlock page : allocatedPages) {
+ if (!loaded || page.getBaseObject() != inMemIterator.getBaseObject()) {
+ released += page.size();
+ freePage(page);
+ } else {
+ lastPage = page;
+ }
+ }
+ allocatedPages.clear();
+ }
+ return released;
+ }
+ }
+
+ @Override
+ public boolean hasNext() {
+ return numRecords > 0;
+ }
+
+ @Override
+ public void loadNext() throws IOException {
+ synchronized (this) {
+ loaded = true;
+ if (nextUpstream != null) {
+ // Just consumed the last record from in memory iterator
+ if (lastPage != null) {
+ freePage(lastPage);
+ lastPage = null;
+ }
+ upstream = nextUpstream;
+ nextUpstream = null;
+
+ assert(inMemSorter != null);
+ long used = inMemSorter.getMemoryUsage();
+ inMemSorter = null;
+ releaseMemory(used);
+ }
+ numRecords--;
+ upstream.loadNext();
+ }
+ }
+
+ @Override
+ public Object getBaseObject() {
+ return upstream.getBaseObject();
+ }
+
+ @Override
+ public long getBaseOffset() {
+ return upstream.getBaseOffset();
+ }
+
+ @Override
+ public int getRecordLength() {
+ return upstream.getRecordLength();
+ }
+
+ @Override
+ public long getKeyPrefix() {
+ return upstream.getKeyPrefix();
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
index 5aad72c..1480f06 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
@@ -70,12 +70,12 @@ public final class UnsafeInMemorySorter {
* Within this buffer, position {@code 2 * i} holds a pointer pointer to the record at
* index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix.
*/
- private long[] pointerArray;
+ private long[] array;
/**
* The position in the sort buffer where new records can be inserted.
*/
- private int pointerArrayInsertPosition = 0;
+ private int pos = 0;
public UnsafeInMemorySorter(
final TaskMemoryManager memoryManager,
@@ -83,37 +83,43 @@ public final class 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;
}
- static long getMemoryRequirementsForPointerArray(long numEntries) {
- return numEntries * 2L * 8L;
+ public long getMemoryUsage() {
+ return array.length * 8L;
}
public boolean hasSpaceForAnotherRecord() {
- return pointerArrayInsertPosition + 2 < pointerArray.length;
+ return pos + 2 <= array.length;
}
public void expandPointerArray() {
- final long[] oldArray = pointerArray;
- // 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);
+ final long[] oldArray = array;
+ array = new long[newLength()];
+ System.arraycopy(oldArray, 0, array, 0, oldArray.length);
}
/**
@@ -127,10 +133,10 @@ public final class UnsafeInMemorySorter {
if (!hasSpaceForAnotherRecord()) {
expandPointerArray();
}
- pointerArray[pointerArrayInsertPosition] = recordPointer;
- pointerArrayInsertPosition++;
- pointerArray[pointerArrayInsertPosition] = keyPrefix;
- pointerArrayInsertPosition++;
+ array[pos] = recordPointer;
+ pos++;
+ array[pos] = keyPrefix;
+ pos++;
}
public static final class SortedIterator extends UnsafeSorterIterator {
@@ -153,11 +159,25 @@ public final class UnsafeInMemorySorter {
this.sortBuffer = sortBuffer;
}
+ public SortedIterator clone () {
+ SortedIterator iter = new SortedIterator(memoryManager, sortBufferInsertPosition, sortBuffer);
+ iter.position = position;
+ iter.baseObject = baseObject;
+ iter.baseOffset = baseOffset;
+ iter.keyPrefix = keyPrefix;
+ iter.recordLength = recordLength;
+ return iter;
+ }
+
@Override
public boolean hasNext() {
return position < sortBufferInsertPosition;
}
+ public int numRecordsLeft() {
+ return (sortBufferInsertPosition - position) / 2;
+ }
+
@Override
public void loadNext() {
// This pointer points to a 4-byte record length, followed by the record's bytes
@@ -187,7 +207,7 @@ public final class UnsafeInMemorySorter {
* {@code next()} will return the same mutable object.
*/
public SortedIterator getSortedIterator() {
- sorter.sort(pointerArray, 0, pointerArrayInsertPosition / 2, sortComparator);
- return new SortedIterator(memoryManager, pointerArrayInsertPosition, pointerArray);
+ sorter.sort(array, 0, pos / 2, sortComparator);
+ return new SortedIterator(memoryManager, pos, array);
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
index 501dfe7..039e940 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
@@ -20,18 +20,18 @@ package org.apache.spark.util.collection.unsafe.sort;
import java.io.*;
import com.google.common.io.ByteStreams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.apache.spark.storage.BlockId;
import org.apache.spark.storage.BlockManager;
import org.apache.spark.unsafe.Platform;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Reads spill files written by {@link UnsafeSorterSpillWriter} (see that class for a description
* of the file format).
*/
-final class UnsafeSorterSpillReader extends UnsafeSorterIterator {
+public final class UnsafeSorterSpillReader extends UnsafeSorterIterator {
private static final Logger logger = LoggerFactory.getLogger(UnsafeSorterSpillReader.class);
private final File file;
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java
index e59a84f..234e211 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java
@@ -35,7 +35,7 @@ import org.apache.spark.unsafe.Platform;
*
* [# of records (int)] [[len (int)][prefix (long)][data (bytes)]...]
*/
-final class UnsafeSorterSpillWriter {
+public final class UnsafeSorterSpillWriter {
static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024;
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
index 6c9a71c..b0cf269 100644
--- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
+++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
@@ -24,6 +24,7 @@ import scala.collection.mutable.ArrayBuffer
import com.google.common.annotations.VisibleForTesting
+import org.apache.spark.util.Utils
import org.apache.spark.{SparkException, TaskContext, SparkConf, Logging}
import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore}
import org.apache.spark.unsafe.array.ByteArrayMethods
@@ -215,8 +216,12 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte
final def releaseExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized {
val curMem = executionMemoryForTask.getOrElse(taskAttemptId, 0L)
if (curMem < numBytes) {
- throw new SparkException(
- s"Internal error: release called on $numBytes bytes but task only has $curMem")
+ if (Utils.isTesting) {
+ throw new SparkException(
+ s"Internal error: release called on $numBytes bytes but task only has $curMem")
+ } else {
+ logWarning(s"Internal error: release called on $numBytes bytes but task only has $curMem")
+ }
}
if (executionMemoryForTask.contains(taskAttemptId)) {
executionMemoryForTask(taskAttemptId) -= numBytes
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
index a76891a..9e00262 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
@@ -78,7 +78,7 @@ private[spark] trait Spillable[C] extends Logging {
if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) {
// Claim up to double our current memory from the shuffle memory pool
val amountToRequest = 2 * currentMemory - myMemoryThreshold
- val granted = taskMemoryManager.acquireExecutionMemory(amountToRequest)
+ val granted = taskMemoryManager.acquireExecutionMemory(amountToRequest, null)
myMemoryThreshold += granted
// If we were granted too little memory to grow further (either tryToAcquire returned 0,
// or we already had more memory than myMemoryThreshold), spill the current collection
@@ -107,7 +107,7 @@ private[spark] trait Spillable[C] extends Logging {
*/
def releaseMemory(): Unit = {
// The amount we requested does not include the initial memory tracking threshold
- taskMemoryManager.releaseExecutionMemory(myMemoryThreshold - initialMemoryThreshold)
+ taskMemoryManager.releaseExecutionMemory(myMemoryThreshold - initialMemoryThreshold, null)
myMemoryThreshold = initialMemoryThreshold
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
index f381db0..dab7b05 100644
--- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
+++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
@@ -17,6 +17,8 @@
package org.apache.spark.memory;
+import java.io.IOException;
+
import org.junit.Assert;
import org.junit.Test;
@@ -25,19 +27,40 @@ import org.apache.spark.unsafe.memory.MemoryBlock;
public class TaskMemoryManagerSuite {
+ class TestMemoryConsumer extends MemoryConsumer {
+ TestMemoryConsumer(TaskMemoryManager memoryManager) {
+ super(memoryManager);
+ }
+
+ @Override
+ public long spill(long size, MemoryConsumer trigger) throws IOException {
+ long used = getUsed();
+ releaseMemory(used);
+ return used;
+ }
+
+ void use(long size) {
+ acquireMemory(size);
+ }
+
+ void free(long size) {
+ releaseMemory(size);
+ }
+ }
+
@Test
public void leakedPageMemoryIsDetected() {
final TaskMemoryManager manager = new TaskMemoryManager(
- new GrantEverythingMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0);
- manager.allocatePage(4096); // leak memory
+ new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0);
+ manager.allocatePage(4096, null); // leak memory
Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory());
}
@Test
public void encodePageNumberAndOffsetOffHeap() {
final TaskMemoryManager manager = new TaskMemoryManager(
- new GrantEverythingMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "true")), 0);
- final MemoryBlock dataPage = manager.allocatePage(256);
+ new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "true")), 0);
+ final MemoryBlock dataPage = manager.allocatePage(256, null);
// In off-heap mode, an offset is an absolute address that may require more than 51 bits to
// encode. This test exercises that corner-case:
final long offset = ((1L << TaskMemoryManager.OFFSET_BITS) + 10);
@@ -49,11 +72,53 @@ public class TaskMemoryManagerSuite {
@Test
public void encodePageNumberAndOffsetOnHeap() {
final TaskMemoryManager manager = new TaskMemoryManager(
- new GrantEverythingMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0);
- final MemoryBlock dataPage = manager.allocatePage(256);
+ new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0);
+ final MemoryBlock dataPage = manager.allocatePage(256, null);
final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, 64);
Assert.assertEquals(dataPage.getBaseObject(), manager.getPage(encodedAddress));
Assert.assertEquals(64, manager.getOffsetInPage(encodedAddress));
}
+ @Test
+ public void cooperativeSpilling() {
+ final TestMemoryManager memoryManager = new TestMemoryManager(new SparkConf());
+ memoryManager.limit(100);
+ final TaskMemoryManager manager = new TaskMemoryManager(memoryManager, 0);
+
+ TestMemoryConsumer c1 = new TestMemoryConsumer(manager);
+ TestMemoryConsumer c2 = new TestMemoryConsumer(manager);
+ c1.use(100);
+ assert(c1.getUsed() == 100);
+ c2.use(100);
+ assert(c2.getUsed() == 100);
+ assert(c1.getUsed() == 0); // spilled
+ c1.use(100);
+ assert(c1.getUsed() == 100);
+ assert(c2.getUsed() == 0); // spilled
+
+ c1.use(50);
+ assert(c1.getUsed() == 50); // spilled
+ assert(c2.getUsed() == 0);
+ c2.use(50);
+ assert(c1.getUsed() == 50);
+ assert(c2.getUsed() == 50);
+
+ c1.use(100);
+ assert(c1.getUsed() == 100);
+ assert(c2.getUsed() == 0); // spilled
+
+ c1.free(20);
+ assert(c1.getUsed() == 80);
+ c2.use(10);
+ assert(c1.getUsed() == 80);
+ assert(c2.getUsed() == 10);
+ c2.use(100);
+ assert(c2.getUsed() == 100);
+ assert(c1.getUsed() == 0); // spilled
+
+ c1.free(0);
+ c2.free(100);
+ assert(manager.cleanUpAllAllocatedMemory() == 0);
+ }
+
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java
index 7fb2f92..9a43f1f 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java
@@ -17,25 +17,29 @@
package org.apache.spark.shuffle.sort;
-import org.apache.spark.shuffle.sort.PackedRecordPointer;
+import java.io.IOException;
+
import org.junit.Test;
-import static org.junit.Assert.*;
import org.apache.spark.SparkConf;
-import org.apache.spark.memory.GrantEverythingMemoryManager;
-import org.apache.spark.unsafe.memory.MemoryBlock;
+import org.apache.spark.memory.TestMemoryManager;
import org.apache.spark.memory.TaskMemoryManager;
-import static org.apache.spark.shuffle.sort.PackedRecordPointer.*;
+import org.apache.spark.unsafe.memory.MemoryBlock;
+
+import static org.apache.spark.shuffle.sort.PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES;
+import static org.apache.spark.shuffle.sort.PackedRecordPointer.MAXIMUM_PARTITION_ID;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
public class PackedRecordPointerSuite {
@Test
- public void heap() {
+ public void heap() throws IOException {
final SparkConf conf = new SparkConf().set("spark.unsafe.offHeap", "false");
final TaskMemoryManager memoryManager =
- new TaskMemoryManager(new GrantEverythingMemoryManager(conf), 0);
- final MemoryBlock page0 = memoryManager.allocatePage(128);
- final MemoryBlock page1 = memoryManager.allocatePage(128);
+ new TaskMemoryManager(new TestMemoryManager(conf), 0);
+ final MemoryBlock page0 = memoryManager.allocatePage(128, null);
+ final MemoryBlock page1 = memoryManager.allocatePage(128, null);
final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1,
page1.getBaseOffset() + 42);
PackedRecordPointer packedPointer = new PackedRecordPointer();
@@ -49,12 +53,12 @@ public class PackedRecordPointerSuite {
}
@Test
- public void offHeap() {
+ public void offHeap() throws IOException {
final SparkConf conf = new SparkConf().set("spark.unsafe.offHeap", "true");
final TaskMemoryManager memoryManager =
- new TaskMemoryManager(new GrantEverythingMemoryManager(conf), 0);
- final MemoryBlock page0 = memoryManager.allocatePage(128);
- final MemoryBlock page1 = memoryManager.allocatePage(128);
+ new TaskMemoryManager(new TestMemoryManager(conf), 0);
+ final MemoryBlock page0 = memoryManager.allocatePage(128, null);
+ final MemoryBlock page1 = memoryManager.allocatePage(128, null);
final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1,
page1.getBaseOffset() + 42);
PackedRecordPointer packedPointer = new PackedRecordPointer();
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java
index 5049a53..2293b1b 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java
@@ -26,7 +26,7 @@ import org.junit.Test;
import org.apache.spark.HashPartitioner;
import org.apache.spark.SparkConf;
import org.apache.spark.unsafe.Platform;
-import org.apache.spark.memory.GrantEverythingMemoryManager;
+import org.apache.spark.memory.TestMemoryManager;
import org.apache.spark.unsafe.memory.MemoryBlock;
import org.apache.spark.memory.TaskMemoryManager;
@@ -60,8 +60,8 @@ public class ShuffleInMemorySorterSuite {
};
final SparkConf conf = new SparkConf().set("spark.unsafe.offHeap", "false");
final TaskMemoryManager memoryManager =
- new TaskMemoryManager(new GrantEverythingMemoryManager(conf), 0);
- final MemoryBlock dataPage = memoryManager.allocatePage(2048);
+ new TaskMemoryManager(new TestMemoryManager(conf), 0);
+ final MemoryBlock dataPage = memoryManager.allocatePage(2048, null);
final Object baseObject = dataPage.getBaseObject();
final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(4);
final HashPartitioner hashPartitioner = new HashPartitioner(4);
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
index d659269..4763395 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
@@ -54,13 +54,14 @@ import org.apache.spark.serializer.*;
import org.apache.spark.scheduler.MapStatus;
import org.apache.spark.shuffle.IndexShuffleBlockResolver;
import org.apache.spark.storage.*;
-import org.apache.spark.memory.GrantEverythingMemoryManager;
+import org.apache.spark.memory.TestMemoryManager;
import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.util.Utils;
public class UnsafeShuffleWriterSuite {
static final int NUM_PARTITITONS = 4;
+ TestMemoryManager memoryManager;
TaskMemoryManager taskMemoryManager;
final HashPartitioner hashPartitioner = new HashPartitioner(NUM_PARTITITONS);
File mergedOutputFile;
@@ -106,10 +107,11 @@ public class UnsafeShuffleWriterSuite {
partitionSizesInMergedFile = null;
spillFilesCreated.clear();
conf = new SparkConf()
- .set("spark.buffer.pageSize", "128m")
+ .set("spark.buffer.pageSize", "1m")
.set("spark.unsafe.offHeap", "false");
taskMetrics = new TaskMetrics();
- taskMemoryManager = new TaskMemoryManager(new GrantEverythingMemoryManager(conf), 0);
+ memoryManager = new TestMemoryManager(conf);
+ taskMemoryManager = new TaskMemoryManager(memoryManager, 0);
when(blockManager.diskBlockManager()).thenReturn(diskBlockManager);
when(blockManager.getDiskWriter(
@@ -344,9 +346,7 @@ public class UnsafeShuffleWriterSuite {
}
assertEquals(sumOfPartitionSizes, mergedOutputFile.length());
- assertEquals(
- HashMultiset.create(dataToWrite),
- HashMultiset.create(readRecordsFromFile()));
+ assertEquals(HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile()));
assertSpillFilesWereCleanedUp();
ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get();
assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten());
@@ -398,20 +398,14 @@ public class UnsafeShuffleWriterSuite {
@Test
public void writeEnoughDataToTriggerSpill() throws Exception {
- taskMemoryManager = spy(taskMemoryManager);
- doCallRealMethod() // initialize sort buffer
- .doCallRealMethod() // allocate initial data page
- .doReturn(0L) // deny request to allocate new page
- .doCallRealMethod() // grant new sort buffer and data page
- .when(taskMemoryManager).acquireExecutionMemory(anyLong());
+ memoryManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES);
final UnsafeShuffleWriter<Object, Object> writer = createWriter(false);
final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<Product2<Object, Object>>();
- final byte[] bigByteArray = new byte[PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES / 128];
- for (int i = 0; i < 128 + 1; i++) {
+ final byte[] bigByteArray = new byte[PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES / 10];
+ for (int i = 0; i < 10 + 1; i++) {
dataToWrite.add(new Tuple2<Object, Object>(i, bigByteArray));
}
writer.write(dataToWrite.iterator());
- verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong());
assertEquals(2, spillFilesCreated.size());
writer.stop(true);
readRecordsFromFile();
@@ -426,19 +420,13 @@ public class UnsafeShuffleWriterSuite {
@Test
public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exception {
- taskMemoryManager = spy(taskMemoryManager);
- doCallRealMethod() // initialize sort buffer
- .doCallRealMethod() // allocate initial data page
- .doReturn(0L) // deny request to allocate new page
- .doCallRealMethod() // grant new sort buffer and data page
- .when(taskMemoryManager).acquireExecutionMemory(anyLong());
+ memoryManager.limit(UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE * 16);
final UnsafeShuffleWriter<Object, Object> writer = createWriter(false);
final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>();
for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE; i++) {
dataToWrite.add(new Tuple2<Object, Object>(i, i));
}
writer.write(dataToWrite.iterator());
- verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong());
assertEquals(2, spillFilesCreated.size());
writer.stop(true);
readRecordsFromFile();
@@ -473,11 +461,11 @@ public class UnsafeShuffleWriterSuite {
final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<Product2<Object, Object>>();
dataToWrite.add(new Tuple2<Object, Object>(1, ByteBuffer.wrap(new byte[1])));
// We should be able to write a record that's right _at_ the max record size
- final byte[] atMaxRecordSize = new byte[writer.maxRecordSizeBytes()];
+ final byte[] atMaxRecordSize = new byte[(int) taskMemoryManager.pageSizeBytes() - 4];
new Random(42).nextBytes(atMaxRecordSize);
dataToWrite.add(new Tuple2<Object, Object>(2, ByteBuffer.wrap(atMaxRecordSize)));
// Inserting a record that's larger than the max record size
- final byte[] exceedsMaxRecordSize = new byte[writer.maxRecordSizeBytes() + 1];
+ final byte[] exceedsMaxRecordSize = new byte[(int) taskMemoryManager.pageSizeBytes()];
new Random(42).nextBytes(exceedsMaxRecordSize);
dataToWrite.add(new Tuple2<Object, Object>(3, ByteBuffer.wrap(exceedsMaxRecordSize)));
writer.write(dataToWrite.iterator());
@@ -524,7 +512,7 @@ public class UnsafeShuffleWriterSuite {
for (int i = 0; i < numRecordsPerPage * 10; i++) {
writer.insertRecordIntoSorter(new Tuple2<Object, Object>(1, 1));
newPeakMemory = writer.getPeakMemoryUsedBytes();
- if (i % numRecordsPerPage == 0 && i != 0) {
+ if (i % numRecordsPerPage == 0) {
// The first page is allocated in constructor, another page will be allocated after
// every numRecordsPerPage records (peak memory should change).
assertEquals(previousPeakMemory + pageSizeBytes, newPeakMemory);
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
index 6e52496..92bd45e 100644
--- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
+++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
@@ -17,40 +17,117 @@
package org.apache.spark.unsafe.map;
-import java.lang.Exception;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.*;
-import org.apache.spark.memory.TaskMemoryManager;
-import org.junit.*;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.junit.Assert.*;
+import scala.Tuple2;
+import scala.Tuple2$;
+import scala.runtime.AbstractFunction1;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
import org.apache.spark.SparkConf;
-import org.apache.spark.memory.GrantEverythingMemoryManager;
-import org.apache.spark.unsafe.array.ByteArrayMethods;
-import org.apache.spark.unsafe.memory.*;
+import org.apache.spark.executor.ShuffleWriteMetrics;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.serializer.SerializerInstance;
+import org.apache.spark.storage.*;
import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.memory.MemoryLocation;
+import org.apache.spark.util.Utils;
+
+import static org.hamcrest.Matchers.greaterThan;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.AdditionalAnswers.returnsSecondArg;
+import static org.mockito.Answers.RETURNS_SMART_NULLS;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.when;
public abstract class AbstractBytesToBytesMapSuite {
private final Random rand = new Random(42);
- private GrantEverythingMemoryManager memoryManager;
+ private TestMemoryManager memoryManager;
private TaskMemoryManager taskMemoryManager;
private final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes
+ final LinkedList<File> spillFilesCreated = new LinkedList<File>();
+ File tempDir;
+
+ @Mock(answer = RETURNS_SMART_NULLS) BlockManager blockManager;
+ @Mock(answer = RETURNS_SMART_NULLS) DiskBlockManager diskBlockManager;
+
+ private static final class CompressStream extends AbstractFunction1<OutputStream, OutputStream> {
+ @Override
+ public OutputStream apply(OutputStream stream) {
+ return stream;
+ }
+ }
+
@Before
public void setup() {
memoryManager =
- new GrantEverythingMemoryManager(
+ new TestMemoryManager(
new SparkConf().set("spark.unsafe.offHeap", "" + useOffHeapMemoryAllocator()));
taskMemoryManager = new TaskMemoryManager(memoryManager, 0);
+
+ tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "unsafe-test");
+ spillFilesCreated.clear();
+ MockitoAnnotations.initMocks(this);
+ when(blockManager.diskBlockManager()).thenReturn(diskBlockManager);
+ when(diskBlockManager.createTempLocalBlock()).thenAnswer(new Answer<Tuple2<TempLocalBlockId, File>>() {
+ @Override
+ public Tuple2<TempLocalBlockId, File> answer(InvocationOnMock invocationOnMock) throws Throwable {
+ TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID());
+ File file = File.createTempFile("spillFile", ".spill", tempDir);
+ spillFilesCreated.add(file);
+ return Tuple2$.MODULE$.apply(blockId, file);
+ }
+ });
+ when(blockManager.getDiskWriter(
+ any(BlockId.class),
+ any(File.class),
+ any(SerializerInstance.class),
+ anyInt(),
+ any(ShuffleWriteMetrics.class))).thenAnswer(new Answer<DiskBlockObjectWriter>() {
+ @Override
+ public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable {
+ Object[] args = invocationOnMock.getArguments();
+
+ return new DiskBlockObjectWriter(
+ (File) args[1],
+ (SerializerInstance) args[2],
+ (Integer) args[3],
+ new CompressStream(),
+ false,
+ (ShuffleWriteMetrics) args[4]
+ );
+ }
+ });
+ when(blockManager.wrapForCompression(any(BlockId.class), any(InputStream.class)))
+ .then(returnsSecondArg());
}
@After
public void tearDown() {
+ Utils.deleteRecursively(tempDir);
+ tempDir = null;
+
Assert.assertEquals(0L, taskMemoryManager.cleanUpAllAllocatedMemory());
if (taskMemoryManager != null) {
long leakedMemory = taskMemoryManager.getMemoryConsumptionForThisTask();
@@ -415,9 +492,8 @@ public abstract class AbstractBytesToBytesMapSuite {
@Test
public void failureToAllocateFirstPage() {
- memoryManager.markExecutionAsOutOfMemory();
+ memoryManager.limit(1024); // longArray
BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 1, PAGE_SIZE_BYTES);
- memoryManager.markExecutionAsOutOfMemory();
try {
final long[] emptyArray = new long[0];
final BytesToBytesMap.Location loc =
@@ -439,7 +515,7 @@ public abstract class AbstractBytesToBytesMapSuite {
int i;
for (i = 0; i < 127; i++) {
if (i > 0) {
- memoryManager.markExecutionAsOutOfMemory();
+ memoryManager.limit(0);
}
final long[] arr = new long[]{i};
final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
@@ -457,6 +533,44 @@ public abstract class AbstractBytesToBytesMapSuite {
}
@Test
+ public void spillInIterator() throws IOException {
+ BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, blockManager, 1, 0.75, 1024, false);
+ try {
+ int i;
+ for (i = 0; i < 1024; i++) {
+ final long[] arr = new long[]{i};
+ final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
+ loc.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
+ }
+ BytesToBytesMap.MapIterator iter = map.iterator();
+ for (i = 0; i < 100; i++) {
+ iter.next();
+ }
+ // Non-destructive iterator is not spillable
+ Assert.assertEquals(0, iter.spill(1024L * 10));
+ for (i = 100; i < 1024; i++) {
+ iter.next();
+ }
+
+ BytesToBytesMap.MapIterator iter2 = map.destructiveIterator();
+ for (i = 0; i < 100; i++) {
+ iter2.next();
+ }
+ Assert.assertTrue(iter2.spill(1024) >= 1024);
+ for (i = 100; i < 1024; i++) {
+ iter2.next();
+ }
+ assertFalse(iter2.hasNext());
+ } finally {
+ map.free();
+ for (File spillFile : spillFilesCreated) {
+ assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up",
+ spillFile.exists());
+ }
+ }
+ }
+
+ @Test
public void initialCapacityBoundsChecking() {
try {
new BytesToBytesMap(taskMemoryManager, 0, PAGE_SIZE_BYTES);
@@ -500,7 +614,7 @@ public abstract class AbstractBytesToBytesMapSuite {
Platform.LONG_ARRAY_OFFSET,
8);
newPeakMemory = map.getPeakMemoryUsedBytes();
- if (i % numRecordsPerPage == 0 && i > 0) {
+ if (i % numRecordsPerPage == 0) {
// We allocated a new page for this record, so peak memory should change
assertEquals(previousPeakMemory + pageSizeBytes, newPeakMemory);
} else {
@@ -519,11 +633,4 @@ public abstract class AbstractBytesToBytesMapSuite {
}
}
- @Test
- public void testAcquirePageInConstructor() {
- final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 1, PAGE_SIZE_BYTES);
- assertEquals(1, map.getNumDataPages());
- map.free();
- }
-
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
index 94d50b9..cfead0e 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
@@ -36,28 +36,29 @@ import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import static org.junit.Assert.*;
-import static org.mockito.AdditionalAnswers.returnsSecondArg;
-import static org.mockito.Answers.RETURNS_SMART_NULLS;
-import static org.mockito.Mockito.*;
import org.apache.spark.SparkConf;
import org.apache.spark.TaskContext;
import org.apache.spark.executor.ShuffleWriteMetrics;
import org.apache.spark.executor.TaskMetrics;
-import org.apache.spark.memory.GrantEverythingMemoryManager;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.serializer.SerializerInstance;
import org.apache.spark.storage.*;
import org.apache.spark.unsafe.Platform;
-import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.util.Utils;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.junit.Assert.*;
+import static org.mockito.AdditionalAnswers.returnsSecondArg;
+import static org.mockito.Answers.RETURNS_SMART_NULLS;
+import static org.mockito.Mockito.*;
+
public class UnsafeExternalSorterSuite {
final LinkedList<File> spillFilesCreated = new LinkedList<File>();
- final GrantEverythingMemoryManager memoryManager =
- new GrantEverythingMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false"));
+ final TestMemoryManager memoryManager =
+ new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false"));
final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(memoryManager, 0);
// Use integer comparison for comparing prefixes (which are partition ids, in this case)
final PrefixComparator prefixComparator = new PrefixComparator() {
@@ -86,7 +87,7 @@ public class UnsafeExternalSorterSuite {
@Mock(answer = RETURNS_SMART_NULLS) TaskContext taskContext;
- private final long pageSizeBytes = new SparkConf().getSizeAsBytes("spark.buffer.pageSize", "64m");
+ private final long pageSizeBytes = new SparkConf().getSizeAsBytes("spark.buffer.pageSize", "4m");
private static final class CompressStream extends AbstractFunction1<OutputStream, OutputStream> {
@Override
@@ -233,7 +234,7 @@ public class UnsafeExternalSorterSuite {
insertNumber(sorter, numRecords - i);
}
assertEquals(1, sorter.getNumberOfAllocatedPages());
- memoryManager.markExecutionAsOutOfMemory();
+ memoryManager.markExecutionAsOutOfMemoryOnce();
// The insertion of this record should trigger a spill:
insertNumber(sorter, 0);
// Ensure that spill files were created
@@ -312,6 +313,62 @@ public class UnsafeExternalSorterSuite {
}
@Test
+ public void forcedSpillingWithReadIterator() throws Exception {
+ final UnsafeExternalSorter sorter = newSorter();
+ long[] record = new long[100];
+ int recordSize = record.length * 8;
+ int n = (int) pageSizeBytes / recordSize * 3;
+ for (int i = 0; i < n; i++) {
+ record[0] = (long) i;
+ sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0);
+ }
+ assert(sorter.getNumberOfAllocatedPages() >= 2);
+ UnsafeExternalSorter.SpillableIterator iter =
+ (UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator();
+ int lastv = 0;
+ for (int i = 0; i < n / 3; i++) {
+ iter.hasNext();
+ iter.loadNext();
+ assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
+ lastv = i;
+ }
+ assert(iter.spill() > 0);
+ assert(iter.spill() == 0);
+ assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == lastv);
+ for (int i = n / 3; i < n; i++) {
+ iter.hasNext();
+ iter.loadNext();
+ assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
+ }
+ sorter.cleanupResources();
+ assertSpillFilesWereCleanedUp();
+ }
+
+ @Test
+ public void forcedSpillingWithNotReadIterator() throws Exception {
+ final UnsafeExternalSorter sorter = newSorter();
+ long[] record = new long[100];
+ int recordSize = record.length * 8;
+ int n = (int) pageSizeBytes / recordSize * 3;
+ for (int i = 0; i < n; i++) {
+ record[0] = (long) i;
+ sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0);
+ }
+ assert(sorter.getNumberOfAllocatedPages() >= 2);
+ UnsafeExternalSorter.SpillableIterator iter =
+ (UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator();
+ assert(iter.spill() > 0);
+ assert(iter.spill() == 0);
+ for (int i = 0; i < n; i++) {
+ iter.hasNext();
+ iter.loadNext();
+ assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
+ }
+ sorter.cleanupResources();
+ assertSpillFilesWereCleanedUp();
+ }
+
+ @Test
public void testPeakMemoryUsed() throws Exception {
final long recordLengthBytes = 8;
final long pageSizeBytes = 256;
@@ -334,7 +391,7 @@ public class UnsafeExternalSorterSuite {
insertNumber(sorter, i);
newPeakMemory = sorter.getPeakMemoryUsedBytes();
// The first page is pre-allocated on instantiation
- if (i % numRecordsPerPage == 0 && i > 0) {
+ if (i % numRecordsPerPage == 0) {
// We allocated a new page for this record, so peak memory should change
assertEquals(previousPeakMemory + pageSizeBytes, newPeakMemory);
} else {
@@ -358,21 +415,5 @@ public class UnsafeExternalSorterSuite {
}
}
- @Test
- public void testReservePageOnInstantiation() throws Exception {
- final UnsafeExternalSorter sorter = newSorter();
- try {
- assertEquals(1, sorter.getNumberOfAllocatedPages());
- // Inserting a new record doesn't allocate more memory since we already have a page
- long peakMemory = sorter.getPeakMemoryUsedBytes();
- insertNumber(sorter, 100);
- assertEquals(peakMemory, sorter.getPeakMemoryUsedBytes());
- assertEquals(1, sorter.getNumberOfAllocatedPages());
- } finally {
- sorter.cleanupResources();
- assertSpillFilesWereCleanedUp();
- }
- }
-
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
index d5de56a..642f658 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
@@ -20,17 +20,19 @@ package org.apache.spark.util.collection.unsafe.sort;
import java.util.Arrays;
import org.junit.Test;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.*;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
import org.apache.spark.HashPartitioner;
import org.apache.spark.SparkConf;
+import org.apache.spark.memory.TestMemoryManager;
+import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.unsafe.Platform;
-import org.apache.spark.memory.GrantEverythingMemoryManager;
import org.apache.spark.unsafe.memory.MemoryBlock;
-import org.apache.spark.memory.TaskMemoryManager;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.isIn;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
public class UnsafeInMemorySorterSuite {
@@ -44,7 +46,7 @@ public class UnsafeInMemorySorterSuite {
public void testSortingEmptyInput() {
final UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(
new TaskMemoryManager(
- new GrantEverythingMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0),
+ new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0),
mock(RecordComparator.class),
mock(PrefixComparator.class),
100);
@@ -66,8 +68,8 @@ public class UnsafeInMemorySorterSuite {
"Mango"
};
final TaskMemoryManager memoryManager = new TaskMemoryManager(
- new GrantEverythingMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0);
- final MemoryBlock dataPage = memoryManager.allocatePage(2048);
+ new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0);
+ final MemoryBlock dataPage = memoryManager.allocatePage(2048, null);
final Object baseObject = dataPage.getBaseObject();
// Write the records into the data page:
long position = dataPage.getBaseOffset();
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/scala/org/apache/spark/FailureSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala
index 0242cbc..203dab9 100644
--- a/core/src/test/scala/org/apache/spark/FailureSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala
@@ -149,7 +149,7 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext {
// cause is preserved
val thrownDueToTaskFailure = intercept[SparkException] {
sc.parallelize(Seq(0)).mapPartitions { iter =>
- TaskContext.get().taskMemoryManager().allocatePage(128)
+ TaskContext.get().taskMemoryManager().allocatePage(128, null)
throw new Exception("intentional task failure")
iter
}.count()
@@ -159,7 +159,7 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext {
// If the task succeeded but memory was leaked, then the task should fail due to that leak
val thrownDueToMemoryLeak = intercept[SparkException] {
sc.parallelize(Seq(0)).mapPartitions { iter =>
- TaskContext.get().taskMemoryManager().allocatePage(128)
+ TaskContext.get().taskMemoryManager().allocatePage(128, null)
iter
}.count()
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala
deleted file mode 100644
index fe102d8..0000000
--- a/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala
+++ /dev/null
@@ -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.memory
-
-import scala.collection.mutable
-
-import org.apache.spark.SparkConf
-import org.apache.spark.storage.{BlockStatus, BlockId}
-
-class GrantEverythingMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = 1) {
- private[memory] override def doAcquireExecutionMemory(
- numBytes: Long,
- evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized {
- if (oom) {
- oom = false
- 0
- } else {
- _executionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory
- numBytes
- }
- }
- override def acquireStorageMemory(
- blockId: BlockId,
- numBytes: Long,
- evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true
- override def acquireUnrollMemory(
- blockId: BlockId,
- numBytes: Long,
- evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true
- override def releaseStorageMemory(numBytes: Long): Unit = { }
- override def maxExecutionMemory: Long = Long.MaxValue
- override def maxStorageMemory: Long = Long.MaxValue
-
- private var oom = false
-
- def markExecutionAsOutOfMemory(): Unit = {
- oom = true
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
index 1265087..4a9479c 100644
--- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
@@ -145,20 +145,20 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
val manager = createMemoryManager(1000L)
val taskMemoryManager = new TaskMemoryManager(manager, 0)
- assert(taskMemoryManager.acquireExecutionMemory(100L) === 100L)
- assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L)
- assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L)
- assert(taskMemoryManager.acquireExecutionMemory(200L) === 100L)
- assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L)
- assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L)
+ assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 100L)
+ assert(taskMemoryManager.acquireExecutionMemory(400L, null) === 400L)
+ assert(taskMemoryManager.acquireExecutionMemory(400L, null) === 400L)
+ assert(taskMemoryManager.acquireExecutionMemory(200L, null) === 100L)
+ assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L)
+ assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L)
- taskMemoryManager.releaseExecutionMemory(500L)
- assert(taskMemoryManager.acquireExecutionMemory(300L) === 300L)
- assert(taskMemoryManager.acquireExecutionMemory(300L) === 200L)
+ taskMemoryManager.releaseExecutionMemory(500L, null)
+ assert(taskMemoryManager.acquireExecutionMemory(300L, null) === 300L)
+ assert(taskMemoryManager.acquireExecutionMemory(300L, null) === 200L)
taskMemoryManager.cleanUpAllAllocatedMemory()
- assert(taskMemoryManager.acquireExecutionMemory(1000L) === 1000L)
- assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L)
+ assert(taskMemoryManager.acquireExecutionMemory(1000L, null) === 1000L)
+ assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L)
}
test("two tasks requesting full execution memory") {
@@ -168,15 +168,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
val futureTimeout: Duration = 20.seconds
// Have both tasks request 500 bytes, then wait until both requests have been granted:
- val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L) }
- val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L) }
+ val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, null) }
+ val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, null) }
assert(Await.result(t1Result1, futureTimeout) === 500L)
assert(Await.result(t2Result1, futureTimeout) === 500L)
// Have both tasks each request 500 bytes more; both should immediately return 0 as they are
// both now at 1 / N
- val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L) }
- val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) }
+ val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, null) }
+ val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) }
assert(Await.result(t1Result2, 200.millis) === 0L)
assert(Await.result(t2Result2, 200.millis) === 0L)
}
@@ -188,15 +188,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
val futureTimeout: Duration = 20.seconds
// Have both tasks request 250 bytes, then wait until both requests have been granted:
- val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L) }
- val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L) }
+ val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, null) }
+ val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, null) }
assert(Await.result(t1Result1, futureTimeout) === 250L)
assert(Await.result(t2Result1, futureTimeout) === 250L)
// Have both tasks each request 500 bytes more.
// We should only grant 250 bytes to each of them on this second request
- val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L) }
- val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) }
+ val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, null) }
+ val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) }
assert(Await.result(t1Result2, futureTimeout) === 250L)
assert(Await.result(t2Result2, futureTimeout) === 250L)
}
@@ -208,17 +208,17 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
val futureTimeout: Duration = 20.seconds
// t1 grabs 1000 bytes and then waits until t2 is ready to make a request.
- val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L) }
+ val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, null) }
assert(Await.result(t1Result1, futureTimeout) === 1000L)
- val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L) }
+ val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, null) }
// Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult
// to make sure the other thread blocks for some time otherwise.
Thread.sleep(300)
- t1MemManager.releaseExecutionMemory(250L)
+ t1MemManager.releaseExecutionMemory(250L, null)
// The memory freed from t1 should now be granted to t2.
assert(Await.result(t2Result1, futureTimeout) === 250L)
// Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory.
- val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L) }
+ val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, null) }
assert(Await.result(t2Result2, 200.millis) === 0L)
}
@@ -229,18 +229,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
val futureTimeout: Duration = 20.seconds
// t1 grabs 1000 bytes and then waits until t2 is ready to make a request.
- val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L) }
+ val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, null) }
assert(Await.result(t1Result1, futureTimeout) === 1000L)
- val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L) }
+ val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, null) }
// Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult
// to make sure the other thread blocks for some time otherwise.
Thread.sleep(300)
// t1 releases all of its memory, so t2 should be able to grab all of the memory
t1MemManager.cleanUpAllAllocatedMemory()
assert(Await.result(t2Result1, futureTimeout) === 500L)
- val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) }
+ val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) }
assert(Await.result(t2Result2, futureTimeout) === 500L)
- val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L) }
+ val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, null) }
assert(Await.result(t2Result3, 200.millis) === 0L)
}
@@ -251,13 +251,13 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
val t2MemManager = new TaskMemoryManager(memoryManager, 2)
val futureTimeout: Duration = 20.seconds
- val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L) }
+ val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, null) }
assert(Await.result(t1Result1, futureTimeout) === 700L)
- val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L) }
+ val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, null) }
assert(Await.result(t2Result1, futureTimeout) === 300L)
- val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L) }
+ val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, null) }
assert(Await.result(t1Result2, 200.millis) === 0L)
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala
new file mode 100644
index 0000000..77e4355
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala
@@ -0,0 +1,70 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.SparkConf
+import org.apache.spark.storage.{BlockStatus, BlockId}
+
+class TestMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = 1) {
+ private[memory] override def doAcquireExecutionMemory(
+ numBytes: Long,
+ evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized {
+ if (oomOnce) {
+ oomOnce = false
+ 0
+ } else if (available >= numBytes) {
+ _executionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory
+ available -= numBytes
+ numBytes
+ } else {
+ _executionMemoryUsed += available
+ val grant = available
+ available = 0
+ grant
+ }
+ }
+ override def acquireStorageMemory(
+ blockId: BlockId,
+ numBytes: Long,
+ evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true
+ override def acquireUnrollMemory(
+ blockId: BlockId,
+ numBytes: Long,
+ evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true
+ override def releaseExecutionMemory(numBytes: Long): Unit = {
+ available += numBytes
+ _executionMemoryUsed -= numBytes
+ }
+ override def releaseStorageMemory(numBytes: Long): Unit = {}
+ override def maxExecutionMemory: Long = Long.MaxValue
+ override def maxStorageMemory: Long = Long.MaxValue
+
+ private var oomOnce = false
+ private var available = Long.MaxValue
+
+ def markExecutionAsOutOfMemoryOnce(): Unit = {
+ oomOnce = true
+ }
+
+ def limit(avail: Long): Unit = {
+ available = avail
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
index 810c74f..f7063d1 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
@@ -96,15 +96,10 @@ final class UnsafeExternalRowSorter {
);
numRowsInserted++;
if (testSpillFrequency > 0 && (numRowsInserted % testSpillFrequency) == 0) {
- spill();
+ sorter.spill();
}
}
- @VisibleForTesting
- void spill() throws IOException {
- sorter.spill();
- }
-
/**
* Return the peak memory used so far, in bytes.
*/
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
index 82c645d..889f970 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
@@ -165,7 +165,7 @@ public final class UnsafeFixedWidthAggregationMap {
public KVIterator<UnsafeRow, UnsafeRow> iterator() {
return new KVIterator<UnsafeRow, UnsafeRow>() {
- private final BytesToBytesMap.BytesToBytesMapIterator mapLocationIterator =
+ private final BytesToBytesMap.MapIterator mapLocationIterator =
map.destructiveIterator();
private final UnsafeRow key = new UnsafeRow();
private final UnsafeRow value = new UnsafeRow();
http://git-wip-us.apache.org/repos/asf/spark/blob/56419cf1/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
index 46301f0..845f2ae 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
@@ -17,13 +17,13 @@
package org.apache.spark.sql.execution;
-import java.io.IOException;
-
import javax.annotation.Nullable;
+import java.io.IOException;
import com.google.common.annotations.VisibleForTesting;
import org.apache.spark.TaskContext;
+import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
import org.apache.spark.sql.catalyst.expressions.codegen.BaseOrdering;
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering;
@@ -33,7 +33,6 @@ import org.apache.spark.unsafe.KVIterator;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.map.BytesToBytesMap;
import org.apache.spark.unsafe.memory.MemoryBlock;
-import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.util.collection.unsafe.sort.*;
/**
@@ -84,18 +83,16 @@ public final class UnsafeKVExternalSorter {
/* initialSize */ 4096,
pageSizeBytes);
} else {
- // Insert the records into the in-memory sorter.
- // We will use the number of elements in the map as the initialSize of the
- // UnsafeInMemorySorter. Because UnsafeInMemorySorter does not accept 0 as the initialSize,
- // we will use 1 as its initial size if the map is empty.
- // TODO: track pointer array memory used by this in-memory sorter! (SPARK-10474)
+ // The memory needed for UnsafeInMemorySorter should be less than longArray in map.
+ map.freeArray();
+ // The memory used by UnsafeInMemorySorter will be counted later (end of this block)
final UnsafeInMemorySorter inMemSorter = new UnsafeInMemorySorter(
taskMemoryManager, recordComparator, prefixComparator, Math.max(1, map.numElements()));
// We cannot use the destructive iterator here because we are reusing the existing memory
// pages in BytesToBytesMap to hold records during sorting.
// The only new memory we are allocating is the pointer/prefix array.
- BytesToBytesMap.BytesToBytesMapIterator iter = map.iterator();
+ BytesToBytesMap.MapIterator iter = map.iterator();
final int numKeyFields = keySchema.size();
UnsafeRow row = new UnsafeRow();
while (iter.hasNext()) {
@@ -117,7 +114,7 @@ public final class UnsafeKVExternalSorter {
}
sorter = UnsafeExternalSorter.createWithExistingInMemorySorter(
- taskContext.taskMemoryManager(),
+ taskMemoryManager,
blockManager,
taskContext,
new KVComparator(ordering, keySchema.length()),
@@ -128,6 +125,8 @@ public final class UnsafeKVExternalSorter {
sorter.spill();
map.free();
+ // counting the memory used UnsafeInMemorySorter
+ taskMemoryManager.acquireExecutionMemory(inMemSorter.getMemoryUsage(), sorter);
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org