diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 7450d27a6d..b602d7cf1c 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -322,8 +322,10 @@ object CometConf extends ShimCometConf { val COMET_COLUMNAR_SHUFFLE_MEMORY_SIZE: OptionalConfigEntry[Long] = conf("spark.comet.columnar.shuffle.memorySize") + .internal() .doc( - "The optional maximum size of the memory used for Comet columnar shuffle, in MiB. " + + "Test-only config. This is only used to test Comet shuffle with Spark tests. " + + "The optional maximum size of the memory used for Comet columnar shuffle, in MiB. " + "Note that this config is only used when `spark.comet.exec.shuffle.mode` is " + "`jvm`. Once allocated memory size reaches this config, the current batch will be " + "flushed to disk immediately. If this is not configured, Comet will use " + @@ -335,8 +337,10 @@ object CometConf extends ShimCometConf { val COMET_COLUMNAR_SHUFFLE_MEMORY_FACTOR: ConfigEntry[Double] = conf("spark.comet.columnar.shuffle.memory.factor") + .internal() .doc( - "Fraction of Comet memory to be allocated per executor process for Comet shuffle. " + + "Test-only config. This is only used to test Comet shuffle with Spark tests. " + + "Fraction of Comet memory to be allocated per executor process for Comet shuffle. " + "Comet memory size is specified by `spark.comet.memoryOverhead` or " + "calculated by `spark.comet.memory.overhead.factor` * `spark.executor.memory`.") .doubleConf @@ -345,6 +349,17 @@ object CometConf extends ShimCometConf { "Ensure that Comet shuffle memory overhead factor is a double greater than 0") .createWithDefault(1.0) + val COMET_COLUMNAR_SHUFFLE_UNIFIED_MEMORY_ALLOCATOR_IN_TEST: ConfigEntry[Boolean] = + conf("spark.comet.columnar.shuffle.unifiedMemoryAllocatorTest") + .doc("Whether to use Spark unified memory allocator for Comet columnar shuffle in tests." + + "If not configured, Comet will use a test-only memory allocator for Comet columnar " + + "shuffle when Spark test env detected. The test-ony allocator is proposed to run with " + + "Spark tests as these tests require on-heap memory configuration. " + + "By default, this config is false.") + .internal() + .booleanConf + .createWithDefault(false) + val COMET_COLUMNAR_SHUFFLE_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.columnar.shuffle.batch.size") .internal() diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index dc97e3c598..69da792223 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -29,7 +29,6 @@ Comet provides the following configuration settings. | spark.comet.columnar.shuffle.async.enabled | Whether to enable asynchronous shuffle for Arrow-based shuffle. | false | | spark.comet.columnar.shuffle.async.max.thread.num | Maximum number of threads on an executor used for Comet async columnar shuffle. This is the upper bound of total number of shuffle threads per executor. In other words, if the number of cores * the number of shuffle threads per task `spark.comet.columnar.shuffle.async.thread.num` is larger than this config. Comet will use this config as the number of shuffle threads per executor instead. | 100 | | spark.comet.columnar.shuffle.async.thread.num | Number of threads used for Comet async columnar shuffle per shuffle task. Note that more threads means more memory requirement to buffer shuffle data before flushing to disk. Also, more threads may not always improve performance, and should be set based on the number of cores available. | 3 | -| spark.comet.columnar.shuffle.memory.factor | Fraction of Comet memory to be allocated per executor process for Comet shuffle. Comet memory size is specified by `spark.comet.memoryOverhead` or calculated by `spark.comet.memory.overhead.factor` * `spark.executor.memory`. | 1.0 | | spark.comet.convert.csv.enabled | When enabled, data from Spark (non-native) CSV v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | | spark.comet.convert.json.enabled | When enabled, data from Spark (non-native) JSON v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | | spark.comet.convert.parquet.enabled | When enabled, data from Spark (non-native) Parquet v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | diff --git a/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java index 2837fa369a..54e349c137 100644 --- a/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java +++ b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java @@ -20,157 +20,75 @@ package org.apache.spark.shuffle.comet; import java.io.IOException; -import java.util.BitSet; import org.apache.spark.SparkConf; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.MemoryMode; -import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.sql.internal.SQLConf; -import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.unsafe.memory.UnsafeMemoryAllocator; +import org.apache.spark.util.Utils; -import org.apache.comet.CometSparkSessionExtensions$; +import org.apache.comet.CometConf$; /** * A simple memory allocator used by `CometShuffleExternalSorter` to allocate memory blocks which - * store serialized rows. We don't rely on Spark memory allocator because we need to allocate - * off-heap memory no matter memory mode is on-heap or off-heap. This allocator is configured with - * fixed size of memory, and it will throw `SparkOutOfMemoryError` if the memory is not enough. - * - *

Some methods are copied from `org.apache.spark.unsafe.memory.TaskMemoryManager` with - * modifications. Most modifications are to remove the dependency on the configured memory mode. + * store serialized rows. This class is simply an implementation of `MemoryConsumer` that delegates + * memory allocation to the `TaskMemoryManager`. This requires that the `TaskMemoryManager` is + * configured with `MemoryMode.OFF_HEAP`, i.e. it is using off-heap memory. */ -public final class CometShuffleMemoryAllocator extends MemoryConsumer { - private final UnsafeMemoryAllocator allocator = new UnsafeMemoryAllocator(); - - private final long pageSize; - private final long totalMemory; - private long allocatedMemory = 0L; - - /** The number of bits used to address the page table. */ - private static final int PAGE_NUMBER_BITS = 13; - - /** The number of entries in the page table. */ - private static final int PAGE_TABLE_SIZE = 1 << PAGE_NUMBER_BITS; - - private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE]; - private final BitSet allocatedPages = new BitSet(PAGE_TABLE_SIZE); +public final class CometShuffleMemoryAllocator extends CometShuffleMemoryAllocatorTrait { + private static CometShuffleMemoryAllocatorTrait INSTANCE; - private static final int OFFSET_BITS = 51; - private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL; - - private static CometShuffleMemoryAllocator INSTANCE; - - public static synchronized CometShuffleMemoryAllocator getInstance( + /** + * Returns the singleton instance of `CometShuffleMemoryAllocator`. This method should be used + * instead of the constructor to ensure that only one instance of `CometShuffleMemoryAllocator` is + * created. For Spark tests, this returns `CometTestShuffleMemoryAllocator` which is a test-only + * allocator that should not be used in production. + */ + public static CometShuffleMemoryAllocatorTrait getInstance( SparkConf conf, TaskMemoryManager taskMemoryManager, long pageSize) { - if (INSTANCE == null) { - INSTANCE = new CometShuffleMemoryAllocator(conf, taskMemoryManager, pageSize); + boolean isSparkTesting = Utils.isTesting(); + boolean useUnifiedMemAllocator = + (boolean) + CometConf$.MODULE$.COMET_COLUMNAR_SHUFFLE_UNIFIED_MEMORY_ALLOCATOR_IN_TEST().get(); + + if (isSparkTesting && !useUnifiedMemAllocator) { + synchronized (CometShuffleMemoryAllocator.class) { + if (INSTANCE == null) { + // CometTestShuffleMemoryAllocator handles pages by itself so it can be a singleton. + INSTANCE = new CometTestShuffleMemoryAllocator(conf, taskMemoryManager, pageSize); + } + } + return INSTANCE; + } else { + if (taskMemoryManager.getTungstenMemoryMode() != MemoryMode.OFF_HEAP) { + throw new IllegalArgumentException( + "CometShuffleMemoryAllocator should be used with off-heap " + + "memory mode, but got " + + taskMemoryManager.getTungstenMemoryMode()); + } + + // CometShuffleMemoryAllocator stores pages in TaskMemoryManager which is not singleton, + // but one instance per task. So we need to create a new instance for each task. + return new CometShuffleMemoryAllocator(taskMemoryManager, pageSize); } - - return INSTANCE; } - CometShuffleMemoryAllocator(SparkConf conf, TaskMemoryManager taskMemoryManager, long pageSize) { + CometShuffleMemoryAllocator(TaskMemoryManager taskMemoryManager, long pageSize) { super(taskMemoryManager, pageSize, MemoryMode.OFF_HEAP); - this.pageSize = pageSize; - this.totalMemory = - CometSparkSessionExtensions$.MODULE$.getCometShuffleMemorySize(conf, SQLConf.get()); - } - - public synchronized long acquireMemory(long size) { - if (allocatedMemory >= totalMemory) { - throw new SparkOutOfMemoryError( - "Unable to acquire " - + size - + " bytes of memory, current usage " - + "is " - + allocatedMemory - + " bytes and max memory is " - + totalMemory - + " bytes"); - } - long allocationSize = Math.min(size, totalMemory - allocatedMemory); - allocatedMemory += allocationSize; - return allocationSize; } public long spill(long l, MemoryConsumer memoryConsumer) throws IOException { + // JVM shuffle writer does not support spilling for other memory consumers return 0; } - public synchronized LongArray allocateArray(long size) { - long required = size * 8L; - MemoryBlock page = allocate(required); - return new LongArray(page); - } - - public synchronized void freeArray(LongArray array) { - if (array == null) { - return; - } - free(array.memoryBlock()); - } - - public synchronized MemoryBlock allocatePage(long required) { - long size = Math.max(pageSize, required); - return allocate(size); - } - - private synchronized MemoryBlock allocate(long required) { - if (required > TaskMemoryManager.MAXIMUM_PAGE_SIZE_BYTES) { - throw new TooLargePageException(required); - } - - long got = acquireMemory(required); - - if (got < required) { - allocatedMemory -= got; - - throw new SparkOutOfMemoryError( - "Unable to acquire " - + required - + " bytes of memory, got " - + got - + " bytes. Available: " - + (totalMemory - allocatedMemory)); - } - - int pageNumber = allocatedPages.nextClearBit(0); - if (pageNumber >= PAGE_TABLE_SIZE) { - allocatedMemory -= got; - - throw new IllegalStateException( - "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); - } - - MemoryBlock block = allocator.allocate(got); - - block.pageNumber = pageNumber; - pageTable[pageNumber] = block; - allocatedPages.set(pageNumber); - - return block; + public synchronized MemoryBlock allocate(long required) { + return this.allocatePage(required); } public synchronized void free(MemoryBlock block) { - if (block.pageNumber == MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) { - // Already freed block - return; - } - allocatedMemory -= block.size(); - - pageTable[block.pageNumber] = null; - allocatedPages.clear(block.pageNumber); - block.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; - - allocator.free(block); - } - - public synchronized long getAvailableMemory() { - return totalMemory - allocatedMemory; + this.freePage(block); } /** @@ -178,21 +96,11 @@ public synchronized long getAvailableMemory() { * method assumes that the page number is valid. */ public long getOffsetInPage(long pagePlusOffsetAddress) { - long offsetInPage = decodeOffset(pagePlusOffsetAddress); - int pageNumber = TaskMemoryManager.decodePageNumber(pagePlusOffsetAddress); - assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); - MemoryBlock page = pageTable[pageNumber]; - assert (page != null); - return page.getBaseOffset() + offsetInPage; - } - - public long decodeOffset(long pagePlusOffsetAddress) { - return pagePlusOffsetAddress & MASK_LONG_LOWER_51_BITS; + return taskMemoryManager.getOffsetInPage(pagePlusOffsetAddress); } public long encodePageNumberAndOffset(int pageNumber, long offsetInPage) { - assert (pageNumber >= 0); - return ((long) pageNumber) << OFFSET_BITS | offsetInPage & MASK_LONG_LOWER_51_BITS; + return TaskMemoryManager.encodePageNumberAndOffset(pageNumber, offsetInPage); } public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { diff --git a/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocatorTrait.java b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocatorTrait.java new file mode 100644 index 0000000000..6831396b3a --- /dev/null +++ b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocatorTrait.java @@ -0,0 +1,41 @@ +/* + * 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.shuffle.comet; + +import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.unsafe.memory.MemoryBlock; + +/** The base class for Comet JVM shuffle memory allocators. */ +public abstract class CometShuffleMemoryAllocatorTrait extends MemoryConsumer { + protected CometShuffleMemoryAllocatorTrait( + TaskMemoryManager taskMemoryManager, long pageSize, MemoryMode mode) { + super(taskMemoryManager, pageSize, mode); + } + + public abstract MemoryBlock allocate(long required); + + public abstract void free(MemoryBlock block); + + public abstract long getOffsetInPage(long pagePlusOffsetAddress); + + public abstract long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage); +} diff --git a/spark/src/main/java/org/apache/spark/shuffle/comet/CometTestShuffleMemoryAllocator.java b/spark/src/main/java/org/apache/spark/shuffle/comet/CometTestShuffleMemoryAllocator.java new file mode 100644 index 0000000000..084e82b2b6 --- /dev/null +++ b/spark/src/main/java/org/apache/spark/shuffle/comet/CometTestShuffleMemoryAllocator.java @@ -0,0 +1,194 @@ +/* + * 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.shuffle.comet; + +import java.io.IOException; +import java.util.BitSet; + +import org.apache.spark.SparkConf; +import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.memory.SparkOutOfMemoryError; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.unsafe.memory.UnsafeMemoryAllocator; + +import org.apache.comet.CometSparkSessionExtensions$; + +/** + * A simple memory allocator used by `CometShuffleExternalSorter` to allocate memory blocks which + * store serialized rows. We don't rely on Spark memory allocator because we need to allocate + * off-heap memory no matter memory mode is on-heap or off-heap. This allocator is configured with + * fixed size of memory, and it will throw `SparkOutOfMemoryError` if the memory is not enough. + * + *

Some methods are copied from `org.apache.spark.unsafe.memory.TaskMemoryManager` with + * modifications. Most modifications are to remove the dependency on the configured memory mode. + * + *

This allocator is test-only and should not be used in production. It is used to test Comet JVM + * shuffle and execution with Spark tests which basically require on-heap memory configuration. + * Thus, this allocator is used to allocate separate off-heap memory allocation for Comet JVM + * shuffle and execution apart from Spark's on-heap memory configuration. + */ +public final class CometTestShuffleMemoryAllocator extends CometShuffleMemoryAllocatorTrait { + private final UnsafeMemoryAllocator allocator = new UnsafeMemoryAllocator(); + + private final long pageSize; + private final long totalMemory; + private long allocatedMemory = 0L; + + /** The number of bits used to address the page table. */ + private static final int PAGE_NUMBER_BITS = 13; + + /** The number of entries in the page table. */ + private static final int PAGE_TABLE_SIZE = 1 << PAGE_NUMBER_BITS; + + private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE]; + private final BitSet allocatedPages = new BitSet(PAGE_TABLE_SIZE); + + private static final int OFFSET_BITS = 51; + private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL; + + private static CometTestShuffleMemoryAllocator INSTANCE; + + CometTestShuffleMemoryAllocator( + SparkConf conf, TaskMemoryManager taskMemoryManager, long pageSize) { + super(taskMemoryManager, pageSize, MemoryMode.OFF_HEAP); + this.pageSize = pageSize; + this.totalMemory = + CometSparkSessionExtensions$.MODULE$.getCometShuffleMemorySize(conf, SQLConf.get()); + } + + private synchronized long _acquireMemory(long size) { + if (allocatedMemory >= totalMemory) { + throw new SparkOutOfMemoryError( + "Unable to acquire " + + size + + " bytes of memory, current usage " + + "is " + + allocatedMemory + + " bytes and max memory is " + + totalMemory + + " bytes"); + } + long allocationSize = Math.min(size, totalMemory - allocatedMemory); + allocatedMemory += allocationSize; + return allocationSize; + } + + public long spill(long l, MemoryConsumer memoryConsumer) throws IOException { + return 0; + } + + public synchronized LongArray allocateArray(long size) { + long required = size * 8L; + MemoryBlock page = allocateMemoryBlock(required); + return new LongArray(page); + } + + public synchronized void freeArray(LongArray array) { + if (array == null) { + return; + } + free(array.memoryBlock()); + } + + public synchronized MemoryBlock allocate(long required) { + long size = Math.max(pageSize, required); + return allocateMemoryBlock(size); + } + + private synchronized MemoryBlock allocateMemoryBlock(long required) { + if (required > TaskMemoryManager.MAXIMUM_PAGE_SIZE_BYTES) { + throw new TooLargePageException(required); + } + + long got = _acquireMemory(required); + + if (got < required) { + allocatedMemory -= got; + + throw new SparkOutOfMemoryError( + "Unable to acquire " + + required + + " bytes of memory, got " + + got + + " bytes. Available: " + + (totalMemory - allocatedMemory)); + } + + int pageNumber = allocatedPages.nextClearBit(0); + if (pageNumber >= PAGE_TABLE_SIZE) { + allocatedMemory -= got; + + throw new IllegalStateException( + "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); + } + + MemoryBlock block = allocator.allocate(got); + + block.pageNumber = pageNumber; + pageTable[pageNumber] = block; + allocatedPages.set(pageNumber); + + return block; + } + + public synchronized void free(MemoryBlock block) { + if (block.pageNumber == MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) { + // Already freed block + return; + } + allocatedMemory -= block.size(); + + pageTable[block.pageNumber] = null; + allocatedPages.clear(block.pageNumber); + block.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; + + allocator.free(block); + } + + /** + * Returns the offset in the page for the given page plus base offset address. Note that this + * method assumes that the page number is valid. + */ + public long getOffsetInPage(long pagePlusOffsetAddress) { + long offsetInPage = decodeOffset(pagePlusOffsetAddress); + int pageNumber = TaskMemoryManager.decodePageNumber(pagePlusOffsetAddress); + assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); + MemoryBlock page = pageTable[pageNumber]; + assert (page != null); + return page.getBaseOffset() + offsetInPage; + } + + public long decodeOffset(long pagePlusOffsetAddress) { + return pagePlusOffsetAddress & MASK_LONG_LOWER_51_BITS; + } + + public long encodePageNumberAndOffset(int pageNumber, long offsetInPage) { + assert (pageNumber >= 0); + return ((long) pageNumber) << OFFSET_BITS | offsetInPage & MASK_LONG_LOWER_51_BITS; + } + + public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { + return encodePageNumberAndOffset(page.pageNumber, offsetInPage - page.getBaseOffset()); + } +} diff --git a/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java b/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java index ed3e2be669..cc44955705 100644 --- a/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java +++ b/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java @@ -38,6 +38,7 @@ import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.comet.CometShuffleChecksumSupport; import org.apache.spark.shuffle.comet.CometShuffleMemoryAllocator; +import org.apache.spark.shuffle.comet.CometShuffleMemoryAllocatorTrait; import org.apache.spark.shuffle.comet.TooLargePageException; import org.apache.spark.sql.comet.execution.shuffle.CometUnsafeShuffleWriter; import org.apache.spark.sql.comet.execution.shuffle.ShuffleThreadPool; @@ -110,7 +111,7 @@ public final class CometShuffleExternalSorter implements CometShuffleChecksumSup // The memory allocator for this sorter. It is used to allocate/free memory pages for this sorter. // Because we need to allocate off-heap memory regardless of configured Spark memory mode // (on-heap/off-heap), we need a separate memory allocator. - private final CometShuffleMemoryAllocator allocator; + private final CometShuffleMemoryAllocatorTrait allocator; /** Whether to write shuffle spilling file in async mode */ private final boolean isAsync; diff --git a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java index f793874d79..dcb9d99d37 100644 --- a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java +++ b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java @@ -41,6 +41,7 @@ import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.comet.CometShuffleMemoryAllocator; +import org.apache.spark.shuffle.comet.CometShuffleMemoryAllocatorTrait; import org.apache.spark.shuffle.sort.RowPartition; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.types.StructType; @@ -87,7 +88,7 @@ public final class CometDiskBlockWriter { static final int MAXIMUM_PAGE_SIZE_BYTES = 1 << 27; /** The Comet allocator used to allocate pages. */ - private final CometShuffleMemoryAllocator allocator; + private final CometShuffleMemoryAllocatorTrait allocator; /** The serializer used to write rows to memory page. */ private final SerializerInstance serializer; @@ -435,12 +436,17 @@ public int compare(CometDiskBlockWriter lhs, CometDiskBlockWriter rhs) { } }); + long totalFreed = 0; for (CometDiskBlockWriter writer : currentWriters) { // Force to spill the writer in a synchronous way, otherwise, we may not be able to // acquire enough memory. + long used = writer.getActiveMemoryUsage(); + writer.doSpill(true); - if (allocator.getAvailableMemory() >= required) { + totalFreed += used; + + if (totalFreed >= required) { break; } } diff --git a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java index cc8c04fdd4..3dc86b05bb 100644 --- a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java +++ b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java @@ -31,7 +31,7 @@ import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; -import org.apache.spark.shuffle.comet.CometShuffleMemoryAllocator; +import org.apache.spark.shuffle.comet.CometShuffleMemoryAllocatorTrait; import org.apache.spark.shuffle.sort.RowPartition; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -62,7 +62,7 @@ public abstract class SpillWriter { // The memory allocator for this sorter. It is used to allocate/free memory pages for this sorter. // Because we need to allocate off-heap memory regardless of configured Spark memory mode // (on-heap/off-heap), we need a separate memory allocator. - protected CometShuffleMemoryAllocator allocator; + protected CometShuffleMemoryAllocatorTrait allocator; protected Native nativeLib; @@ -134,7 +134,7 @@ public boolean acquireNewPageIfNecessary(int required) { || pageCursor + required > currentPage.getBaseOffset() + currentPage.size()) { // TODO: try to find space in previous pages try { - currentPage = allocator.allocatePage(required); + currentPage = allocator.allocate(required); } catch (SparkOutOfMemoryError error) { try { // Cannot allocate enough memory, spill @@ -155,7 +155,7 @@ public boolean acquireNewPageIfNecessary(int required) { public void initialCurrentPage(int required) { assert (currentPage == null); try { - currentPage = allocator.allocatePage(required); + currentPage = allocator.allocate(required); } catch (SparkOutOfMemoryError e) { logger.error("Unable to acquire {} bytes of memory", required); throw e; diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 952ef39e90..6a5c0efea7 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1162,8 +1162,22 @@ object CometSparkSessionExtensions extends Logging { } } + private[comet] def isOffHeapEnabled(conf: SQLConf): Boolean = + conf.contains("spark.memory.offHeap.enabled") && + conf.getConfString("spark.memory.offHeap.enabled").toBoolean + + // Copied from org.apache.spark.util.Utils which is private to Spark. + private[comet] def isTesting: Boolean = { + System.getenv("SPARK_TESTING") != null || System.getProperty("spark.testing") != null + } + + // Check whether Comet shuffle is enabled: + // 1. `COMET_EXEC_SHUFFLE_ENABLED` is true + // 2. `spark.shuffle.manager` is set to `CometShuffleManager` + // 3. Off-heap memory is enabled || Spark/Comet unit testing private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean = - COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) + COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) && + (isOffHeapEnabled(conf) || isTesting) private[comet] def getCometShuffleNotEnabledReason(conf: SQLConf): Option[String] = { if (!COMET_EXEC_SHUFFLE_ENABLED.get(conf)) { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala index ecc056ddda..a2c2f66517 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala @@ -40,6 +40,7 @@ import org.apache.comet.CometConf abstract class CometColumnarShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper { protected val adaptiveExecutionEnabled: Boolean protected val numElementsForceSpillThreshold: Int = 10 + protected val useUnifiedMemoryAllocator: Boolean = true override protected def sparkConf: SparkConf = { val conf = super.sparkConf @@ -57,6 +58,8 @@ abstract class CometColumnarShuffleSuite extends CometTestBase with AdaptiveSpar CometConf.COMET_COLUMNAR_SHUFFLE_SPILL_THRESHOLD.key -> numElementsForceSpillThreshold.toString, CometConf.COMET_EXEC_ENABLED.key -> "false", CometConf.COMET_SHUFFLE_MODE.key -> "jvm", + CometConf.COMET_COLUMNAR_SHUFFLE_UNIFIED_MEMORY_ALLOCATOR_IN_TEST.key -> + useUnifiedMemoryAllocator.toString, CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_COLUMNAR_SHUFFLE_MEMORY_SIZE.key -> "1536m") { testFun @@ -968,6 +971,13 @@ abstract class CometColumnarShuffleSuite extends CometTestBase with AdaptiveSpar } } +class CometTestMemoryAllocatorShuffleSuite extends CometColumnarShuffleSuite { + override protected val asyncShuffleEnable: Boolean = false + override protected val adaptiveExecutionEnabled: Boolean = true + // Explicitly test with `CometTestShuffleMemoryAllocator` + override protected val useUnifiedMemoryAllocator: Boolean = false +} + class CometAsyncShuffleSuite extends CometColumnarShuffleSuite { override protected val asyncShuffleEnable: Boolean = true