/** * Checks whether there is enough space to insert an additional record in to the sort pointer * array and grows the array if additional space is required. If the required space cannot be * obtained, then the in-memory data will be spilled to disk. */ private void growPointerArrayIfNecessary() throws IOException { assert(inMemSorter != null); if (!inMemSorter.hasSpaceForAnotherRecord()) { long used = inMemSorter.getMemoryUsage(); LongArray array; try { // could trigger spilling array = allocateArray(used / 8 * 2); } catch (OutOfMemoryError e) { // should have trigger spilling if (!inMemSorter.hasSpaceForAnotherRecord()) { logger.error("Unable to grow the pointer array"); throw e; } return; } // check if spilling is triggered or not if (inMemSorter.hasSpaceForAnotherRecord()) { freeArray(array); } else { inMemSorter.expandPointerArray(array); } } }
/** * Force all memory and spill files to be deleted; called by shuffle error-handling code. */ public void cleanupResources() { freeMemory(); if (inMemSorter != null) { inMemSorter.free(); inMemSorter = null; } for (SpillInfo spill : spills) { if (spill.file.exists() && !spill.file.delete()) { logger.error("Unable to delete spill file {}", spill.file.getPath()); } } }
ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize, boolean useRadixSort) { this.consumer = consumer; assert (initialSize > 0); this.initialSize = initialSize; this.useRadixSort = useRadixSort; this.array = consumer.allocateArray(initialSize); this.usableCapacity = getUsableCapacity(); }
/** * Sort and spill the current records in response to memory pressure. */ @Override public long spill(long size, MemoryConsumer trigger) throws IOException { if (trigger != this || inMemSorter == null || inMemSorter.numRecords() == 0) { return 0L; } logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", Thread.currentThread().getId(), Utils.bytesToString(getMemoryUsage()), spills.size(), spills.size() > 1 ? " times" : " time"); writeSortedFile(false); final long spillSize = freeMemory(); inMemSorter.reset(); // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the // records. Otherwise, if the task is over allocated memory, then without freeing the memory // pages, we might not be able to get memory for the pointer array. taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); return spillSize; }
@Test public void testSortingManyNumbers() throws Exception { ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(consumer, 4, shouldUseRadixSort()); int[] numbersToSort = new int[128000]; Random random = new Random(16); for (int i = 0; i < numbersToSort.length; i++) { if (!sorter.hasSpaceForAnotherRecord()) { sorter.expandPointerArray(consumer.allocateArray(sorter.getMemoryUsage() / 8 * 2)); } numbersToSort[i] = random.nextInt(PackedRecordPointer.MAXIMUM_PARTITION_ID + 1); sorter.insertRecord(0, numbersToSort[i]); } Arrays.sort(numbersToSort); int[] sorterResult = new int[numbersToSort.length]; ShuffleInMemorySorter.ShuffleSorterIterator iter = sorter.getSortedIterator(); int j = 0; while (iter.hasNext()) { iter.loadNext(); sorterResult[j] = iter.packedRecordPointer.getPartitionId(); j += 1; } Assert.assertArrayEquals(numbersToSort, sorterResult); } }
@Test public void testSortingEmptyInput() { final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter( consumer, 100, shouldUseRadixSort()); final ShuffleInMemorySorter.ShuffleSorterIterator iter = sorter.getSortedIterator(); Assert.assertFalse(iter.hasNext()); }
/** * Write a record to the shuffle sorter. */ public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId) throws IOException { // for tests assert(inMemSorter != null); if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { logger.info("Spilling data because number of spilledRecords crossed the threshold " + numElementsForSpillThreshold); spill(); } growPointerArrayIfNecessary(); // Need 4 bytes to store the record length. final int required = length + 4; acquireNewPageIfNecessary(required); assert(currentPage != null); 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; inMemSorter.insertRecord(recordAddress, partitionId); }
private long getMemoryUsage() { long totalPageSize = 0; for (MemoryBlock page : allocatedPages) { totalPageSize += page.size(); } return ((inMemSorter == null) ? 0 : inMemSorter.getMemoryUsage()) + totalPageSize; }
/** * Inserts a record to be sorted. * * @param recordPointer a pointer to the record, encoded by the task memory manager. Due to * certain pointer compression techniques used by the sorter, the sort can * only operate on pointers that point to locations in the first * {@link PackedRecordPointer#MAXIMUM_PAGE_SIZE_BYTES} bytes of a data page. * @param partitionId the partition id, which must be less than or equal to * {@link PackedRecordPointer#MAXIMUM_PARTITION_ID}. */ public void insertRecord(long recordPointer, int partitionId) { if (!hasSpaceForAnotherRecord()) { throw new IllegalStateException("There is no space for new record"); } array.set(pos, PackedRecordPointer.packPointer(recordPointer, partitionId)); pos++; }
ShuffleExternalSorter( TaskMemoryManager memoryManager, BlockManager blockManager, TaskContext taskContext, int initialSize, int numPartitions, SparkConf conf, ShuffleWriteMetrics writeMetrics) { super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), memoryManager.getTungstenMemoryMode()); this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; 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", 1024 * 1024 * 1024); this.writeMetrics = writeMetrics; this.inMemSorter = new ShuffleInMemorySorter( this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); this.peakMemoryUsedBytes = getMemoryUsage(); }
inMemSorter.getSortedIterator();
@Test public void testSortingManyNumbers() throws Exception { ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(consumer, 4, shouldUseRadixSort()); int[] numbersToSort = new int[128000]; Random random = new Random(16); for (int i = 0; i < numbersToSort.length; i++) { if (!sorter.hasSpaceForAnotherRecord()) { sorter.expandPointerArray(consumer.allocateArray(sorter.getMemoryUsage() / 8 * 2)); } numbersToSort[i] = random.nextInt(PackedRecordPointer.MAXIMUM_PARTITION_ID + 1); sorter.insertRecord(0, numbersToSort[i]); } Arrays.sort(numbersToSort); int[] sorterResult = new int[numbersToSort.length]; ShuffleInMemorySorter.ShuffleSorterIterator iter = sorter.getSortedIterator(); int j = 0; while (iter.hasNext()) { iter.loadNext(); sorterResult[j] = iter.packedRecordPointer.getPartitionId(); j += 1; } Assert.assertArrayEquals(numbersToSort, sorterResult); } }
@Test public void testSortingEmptyInput() { final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter( consumer, 100, shouldUseRadixSort()); final ShuffleInMemorySorter.ShuffleSorterIterator iter = sorter.getSortedIterator(); Assert.assertFalse(iter.hasNext()); }
/** * Write a record to the shuffle sorter. */ public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId) throws IOException { // for tests assert(inMemSorter != null); if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { logger.info("Spilling data because number of spilledRecords crossed the threshold " + numElementsForSpillThreshold); spill(); } growPointerArrayIfNecessary(); final int uaoSize = UnsafeAlignedOffset.getUaoSize(); // Need 4 or 8 bytes to store the record length. final int required = length + uaoSize; acquireNewPageIfNecessary(required); assert(currentPage != null); final Object base = currentPage.getBaseObject(); final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); UnsafeAlignedOffset.putSize(base, pageCursor, length); pageCursor += uaoSize; Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length); pageCursor += length; inMemSorter.insertRecord(recordAddress, partitionId); }
private long getMemoryUsage() { long totalPageSize = 0; for (MemoryBlock page : allocatedPages) { totalPageSize += page.size(); } return ((inMemSorter == null) ? 0 : inMemSorter.getMemoryUsage()) + totalPageSize; }
/** * Inserts a record to be sorted. * * @param recordPointer a pointer to the record, encoded by the task memory manager. Due to * certain pointer compression techniques used by the sorter, the sort can * only operate on pointers that point to locations in the first * {@link PackedRecordPointer#MAXIMUM_PAGE_SIZE_BYTES} bytes of a data page. * @param partitionId the partition id, which must be less than or equal to * {@link PackedRecordPointer#MAXIMUM_PARTITION_ID}. */ public void insertRecord(long recordPointer, int partitionId) { if (!hasSpaceForAnotherRecord()) { throw new IllegalStateException("There is no space for new record"); } array.set(pos, PackedRecordPointer.packPointer(recordPointer, partitionId)); pos++; }
/** * Sort and spill the current records in response to memory pressure. */ @Override public long spill(long size, MemoryConsumer trigger) throws IOException { if (trigger != this || inMemSorter == null || inMemSorter.numRecords() == 0) { return 0L; } logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", Thread.currentThread().getId(), Utils.bytesToString(getMemoryUsage()), spills.size(), spills.size() > 1 ? " times" : " time"); writeSortedFile(false); final long spillSize = freeMemory(); inMemSorter.reset(); // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the // records. Otherwise, if the task is over allocated memory, then without freeing the memory // pages, we might not be able to get memory for the pointer array. taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); return spillSize; }
ShuffleExternalSorter( TaskMemoryManager memoryManager, BlockManager blockManager, TaskContext taskContext, int initialSize, int numPartitions, SparkConf conf, ShuffleWriteMetrics writeMetrics) { super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), memoryManager.getTungstenMemoryMode()); this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; this.numPartitions = numPartitions; // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSizeBytes = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.numElementsForSpillThreshold = (int) conf.get(package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()); this.writeMetrics = writeMetrics; this.inMemSorter = new ShuffleInMemorySorter( this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); }
inMemSorter.getSortedIterator();
@Test public void testSortingManyNumbers() throws Exception { ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(consumer, 4, shouldUseRadixSort()); int[] numbersToSort = new int[128000]; Random random = new Random(16); for (int i = 0; i < numbersToSort.length; i++) { if (!sorter.hasSpaceForAnotherRecord()) { sorter.expandPointerArray(consumer.allocateArray(sorter.getMemoryUsage() / 8 * 2)); } numbersToSort[i] = random.nextInt(PackedRecordPointer.MAXIMUM_PARTITION_ID + 1); sorter.insertRecord(0, numbersToSort[i]); } Arrays.sort(numbersToSort); int[] sorterResult = new int[numbersToSort.length]; ShuffleInMemorySorter.ShuffleSorterIterator iter = sorter.getSortedIterator(); int j = 0; while (iter.hasNext()) { iter.loadNext(); sorterResult[j] = iter.packedRecordPointer.getPartitionId(); j += 1; } Assert.assertArrayEquals(numbersToSort, sorterResult); } }