Skip to content

Commit

Permalink
Refactor and document logic that decides when to spill.
Browse files Browse the repository at this point in the history
  • Loading branch information
JoshRosen committed May 7, 2015
1 parent 1ce1300 commit b95e642
Showing 1 changed file with 60 additions and 37 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@ public final class UnsafeShuffleExternalSorter {
private UnsafeShuffleSorter sorter;
private MemoryBlock currentPage = null;
private long currentPagePosition = -1;
private long freeSpaceInCurrentPage = 0;

public UnsafeShuffleExternalSorter(
TaskMemoryManager memoryManager,
Expand Down Expand Up @@ -245,22 +246,40 @@ private long freeMemory() {
allocatedPages.clear();
currentPage = null;
currentPagePosition = -1;
freeSpaceInCurrentPage = 0;
return memoryFreed;
}

/**
* Checks whether there is enough space to insert a new record into the sorter. If there is
* insufficient space, either allocate more memory or spill the current sort data (if spilling
* is enabled), then insert the record.
* Checks whether there is enough space to insert a new record into the sorter.
*
* @param requiredSpace the required space in the data page, in bytes, including space for storing
* the record size.
* @return true if the record can be inserted without requiring more allocations, false otherwise.
*/
private boolean haveSpaceForRecord(int requiredSpace) {
logger.warn("Seeing if there's space for the record");
assert (requiredSpace > 0);
// The sort array will automatically expand when inserting a new record, so we only need to
// worry about it having free space when spilling is enabled.
final boolean sortBufferHasSpace = !spillingEnabled || sorter.hasSpaceForAnotherRecord();
final boolean dataPageHasSpace = requiredSpace <= freeSpaceInCurrentPage;
return (sortBufferHasSpace && dataPageHasSpace);
}

/**
* Allocates more memory in order to insert an additional record. If spilling is enabled, this
* will request additional memory from the {@link ShuffleMemoryManager} and spill if the requested
* memory can not be obtained. If spilling is disabled, then this will allocate memory without
* coordinating with the ShuffleMemoryManager.
*
* @param requiredSpace the required space in the data page, in bytes, including space for storing
* the record size.
*/
private void ensureSpaceInDataPage(int requiredSpace) throws IOException {
// TODO: we should re-order the `if` cases in this function so that the most common case (there
// is enough space) appears first.

// TODO: merge these steps to first calculate total memory requirements for this insert,
// then try to acquire; no point in acquiring sort buffer only to spill due to no space in the
// data page.
if (!sorter.hasSpaceForAnotherRecord() && spillingEnabled) {
private void allocateSpaceForRecord(int requiredSpace) throws IOException {
if (spillingEnabled && !sorter.hasSpaceForAnotherRecord()) {
logger.debug("Attempting to expand sort buffer");
final long oldSortBufferMemoryUsage = sorter.getMemoryUsage();
final long memoryToGrowSortBuffer = oldSortBufferMemoryUsage * 2;
final long memoryAcquired = shuffleMemoryManager.tryToAcquire(memoryToGrowSortBuffer);
Expand All @@ -272,33 +291,33 @@ private void ensureSpaceInDataPage(int requiredSpace) throws IOException {
shuffleMemoryManager.release(oldSortBufferMemoryUsage);
}
}

final long spaceInCurrentPage;
if (currentPage != null) {
spaceInCurrentPage = PAGE_SIZE - (currentPagePosition - currentPage.getBaseOffset());
} else {
spaceInCurrentPage = 0;
}
if (requiredSpace > PAGE_SIZE) {
// TODO: throw a more specific exception?
throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
PAGE_SIZE + ")");
} else if (requiredSpace > spaceInCurrentPage) {
if (spillingEnabled) {
final long memoryAcquired = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
if (memoryAcquired < PAGE_SIZE) {
shuffleMemoryManager.release(memoryAcquired);
spill();
final long memoryAcquiredAfterSpill = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
if (memoryAcquiredAfterSpill != PAGE_SIZE) {
shuffleMemoryManager.release(memoryAcquiredAfterSpill);
throw new IOException("Can't allocate memory!");
if (requiredSpace > freeSpaceInCurrentPage) {
logger.debug("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 > PAGE_SIZE) {
throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
PAGE_SIZE + ")");
} else {
if (spillingEnabled) {
final long memoryAcquired = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
if (memoryAcquired < PAGE_SIZE) {
shuffleMemoryManager.release(memoryAcquired);
spill();
final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
if (memoryAcquiredAfterSpilling != PAGE_SIZE) {
shuffleMemoryManager.release(memoryAcquiredAfterSpilling);
throw new IOException("Can't allocate memory!");
}
}
}
currentPage = memoryManager.allocatePage(PAGE_SIZE);
currentPagePosition = currentPage.getBaseOffset();
freeSpaceInCurrentPage = PAGE_SIZE;
allocatedPages.add(currentPage);
}
currentPage = memoryManager.allocatePage(PAGE_SIZE);
currentPagePosition = currentPage.getBaseOffset();
allocatedPages.add(currentPage);
}
}

Expand All @@ -311,21 +330,25 @@ public void insertRecord(
int lengthInBytes,
int partitionId) throws IOException {
// Need 4 bytes to store the record length.
ensureSpaceInDataPage(lengthInBytes + 4);
final int totalSpaceRequired = lengthInBytes + 4;
if (!haveSpaceForRecord(totalSpaceRequired)) {
allocateSpaceForRecord(totalSpaceRequired);
}

final long recordAddress =
memoryManager.encodePageNumberAndOffset(currentPage, currentPagePosition);
final Object dataPageBaseObject = currentPage.getBaseObject();
PlatformDependent.UNSAFE.putInt(dataPageBaseObject, currentPagePosition, lengthInBytes);
currentPagePosition += 4;
freeSpaceInCurrentPage -= 4;
PlatformDependent.copyMemory(
recordBaseObject,
recordBaseOffset,
dataPageBaseObject,
currentPagePosition,
lengthInBytes);
currentPagePosition += lengthInBytes;

freeSpaceInCurrentPage -= lengthInBytes;
sorter.insertRecord(recordAddress, partitionId);
}

Expand Down

0 comments on commit b95e642

Please sign in to comment.