From c32bf0c949e7ab4a99e5a98d07639812267c97ee Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 13 Nov 2024 22:18:56 -0800 Subject: [PATCH 01/47] chore: Simplify CometShuffleMemoryAllocator to use Spark unified memory allocator (#1063) --- .../scala/org/apache/comet/CometConf.scala | 19 +- docs/source/user-guide/configs.md | 1 - .../comet/CometShuffleMemoryAllocator.java | 182 ++++------------ .../CometShuffleMemoryAllocatorTrait.java | 41 ++++ .../CometTestShuffleMemoryAllocator.java | 194 ++++++++++++++++++ .../sort/CometShuffleExternalSorter.java | 3 +- .../shuffle/CometDiskBlockWriter.java | 10 +- .../comet/execution/shuffle/SpillWriter.java | 8 +- .../comet/CometSparkSessionExtensions.scala | 16 +- .../exec/CometColumnarShuffleSuite.scala | 10 + 10 files changed, 336 insertions(+), 148 deletions(-) create mode 100644 spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocatorTrait.java create mode 100644 spark/src/main/java/org/apache/spark/shuffle/comet/CometTestShuffleMemoryAllocator.java 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 From f3da8449b90cfb7d7497db43a8ee57cc0d9c0a54 Mon Sep 17 00:00:00 2001 From: Raz Luvaton Date: Thu, 14 Nov 2024 18:10:13 +0200 Subject: [PATCH 02/47] docs: Update benchmarking.md (#1085) --- docs/source/contributor-guide/benchmarking.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/contributor-guide/benchmarking.md b/docs/source/contributor-guide/benchmarking.md index 8c8d53e677..08aa10abc5 100644 --- a/docs/source/contributor-guide/benchmarking.md +++ b/docs/source/contributor-guide/benchmarking.md @@ -22,7 +22,7 @@ under the License. To track progress on performance, we regularly run benchmarks derived from TPC-H and TPC-DS. Data generation and benchmarking documentation and scripts are available in the [DataFusion Benchmarks](https://github.com/apache/datafusion-benchmarks) GitHub repository. -We also have many micro benchmarks that can be run from an IDE located [here]()https://github.com/apache/datafusion-comet/tree/main/spark/src/test/scala/org/apache/spark/sql/benchmark). +We also have many micro benchmarks that can be run from an IDE located [here](https://github.com/apache/datafusion-comet/tree/main/spark/src/test/scala/org/apache/spark/sql/benchmark). Here are example commands for running the benchmarks against a Spark cluster. This command will need to be adapted based on the Spark environment and location of data files. From 2c832b4a56eafa3dacbe3ef31d99adabccb803bf Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Nov 2024 12:55:57 -0700 Subject: [PATCH 03/47] feat: Require offHeap memory to be enabled (always use unified memory) (#1062) * Require offHeap memory * remove unused import * use off heap memory in stability tests * reorder imports --- docs/source/user-guide/tuning.md | 32 ++----------------- native/core/src/execution/jni_api.rs | 24 ++------------ .../org/apache/comet/CometExecIterator.scala | 11 +------ .../comet/CometSparkSessionExtensions.scala | 7 ++++ .../sql/comet/CometPlanStabilitySuite.scala | 3 ++ 5 files changed, 16 insertions(+), 61 deletions(-) diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index 30ada4c9b1..b1838ca801 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -23,40 +23,12 @@ Comet provides some tuning options to help you get the best performance from you ## Memory Tuning -Comet provides two options for memory management: - -- **Unified Memory Management** shares an off-heap memory pool between Spark and Comet. This is the recommended option. -- **Native Memory Management** leverages DataFusion's memory management for the native plans and allocates memory independently of Spark. - -### Unified Memory Management - -This option is automatically enabled when `spark.memory.offHeap.enabled=true`. +Comet shares an off-heap memory pool between Spark and Comet. This requires setting `spark.memory.offHeap.enabled=true`. +If this setting is not enabled, Comet will not accelerate queries and will fall back to Spark. Each executor will have a single memory pool which will be shared by all native plans being executed within that process, and by Spark itself. The size of the pool is specified by `spark.memory.offHeap.size`. -### Native Memory Management - -This option is automatically enabled when `spark.memory.offHeap.enabled=false`. - -Each native plan has a dedicated memory pool. - -By default, the size of each pool is `spark.comet.memory.overhead.factor * spark.executor.memory`. The default value -for `spark.comet.memory.overhead.factor` is `0.2`. - -It is important to take executor concurrency into account. The maximum number of concurrent plans in an executor can -be calculated with `spark.executor.cores / spark.task.cpus`. - -For example, if the executor can execute 4 plans concurrently, then the total amount of memory allocated will be -`4 * spark.comet.memory.overhead.factor * spark.executor.memory`. - -It is also possible to set `spark.comet.memoryOverhead` to the desired size for each pool, rather than calculating -it based on `spark.comet.memory.overhead.factor`. - -If both `spark.comet.memoryOverhead` and `spark.comet.memory.overhead.factor` are set, the former will be used. - -Comet will allocate at least `spark.comet.memory.overhead.min` memory per pool. - ### Determining How Much Memory to Allocate Generally, increasing memory overhead will improve query performance, especially for queries containing joins and diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index d7e8ccaba1..47d87fe1af 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -202,27 +202,9 @@ fn prepare_datafusion_session_context( let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); - // Check if we are using unified memory manager integrated with Spark. Default to false if not - // set. - let use_unified_memory_manager = parse_bool(conf, "use_unified_memory_manager")?; - - if use_unified_memory_manager { - // Set Comet memory pool for native - let memory_pool = CometMemoryPool::new(comet_task_memory_manager); - rt_config = rt_config.with_memory_pool(Arc::new(memory_pool)); - } else { - // Use the memory pool from DF - if conf.contains_key("memory_limit") { - let memory_limit = conf.get("memory_limit").unwrap().parse::()?; - let memory_fraction = conf - .get("memory_fraction") - .ok_or(CometError::Internal( - "Config 'memory_fraction' is not specified from Comet JVM side".to_string(), - ))? - .parse::()?; - rt_config = rt_config.with_memory_limit(memory_limit, memory_fraction) - } - } + // Set Comet memory pool for native + let memory_pool = CometMemoryPool::new(comet_task_memory_manager); + rt_config = rt_config.with_memory_pool(Arc::new(memory_pool)); // Get Datafusion configuration from Spark Execution context // can be configured in Comet Spark JVM using Spark --conf parameters diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 07dd80c39e..b1f22726a6 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -23,7 +23,7 @@ import org.apache.spark._ import org.apache.spark.sql.comet.CometMetricNode import org.apache.spark.sql.vectorized._ -import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} +import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} import org.apache.comet.vector.NativeUtil /** @@ -75,15 +75,6 @@ class CometExecIterator( val result = new java.util.HashMap[String, String]() val conf = SparkEnv.get.conf - val maxMemory = CometSparkSessionExtensions.getCometMemoryOverhead(conf) - // Only enable unified memory manager when off-heap mode is enabled. Otherwise, - // we'll use the built-in memory pool from DF, and initializes with `memory_limit` - // and `memory_fraction` below. - result.put( - "use_unified_memory_manager", - String.valueOf(conf.get("spark.memory.offHeap.enabled", "false"))) - result.put("memory_limit", String.valueOf(maxMemory)) - result.put("memory_fraction", String.valueOf(COMET_EXEC_MEMORY_FRACTION.get())) result.put("batch_size", String.valueOf(COMET_BATCH_SIZE.get())) result.put("debug_native", String.valueOf(COMET_DEBUG_ENABLED.get())) result.put("explain_native", String.valueOf(COMET_EXPLAIN_NATIVE_ENABLED.get())) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 6a5c0efea7..1c4ffcf3e5 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -915,6 +915,13 @@ class CometSparkSessionExtensions } override def apply(plan: SparkPlan): SparkPlan = { + + // Comet required off-heap memory to be enabled + if ("true" != conf.getConfString("spark.memory.offHeap.enabled", "false")) { + logInfo("Comet extension disabled because spark.memory.offHeap.enabled=false") + return plan + } + // DataFusion doesn't have ANSI mode. For now we just disable CometExec if ANSI mode is // enabled. if (isANSIEnabled(conf)) { diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index a553e61c78..16a7e5338d 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -26,6 +26,7 @@ import scala.collection.mutable import org.apache.commons.io.FileUtils import org.apache.spark.SparkContext +import org.apache.spark.internal.config.{MEMORY_OFFHEAP_ENABLED, MEMORY_OFFHEAP_SIZE} import org.apache.spark.sql.TPCDSBase import org.apache.spark.sql.catalyst.expressions.AttributeSet import org.apache.spark.sql.catalyst.util.resourceToString @@ -286,6 +287,8 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa conf.set( "spark.shuffle.manager", "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") + conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") conf.set(CometConf.COMET_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "1g") From 7cec28588345c2d240119e0500e2cacc1fd8a788 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 14 Nov 2024 16:47:57 -0800 Subject: [PATCH 04/47] test: Restore one test in CometExecSuite by adding COMET_SHUFFLE_MODE config (#1087) --- .../apache/comet/exec/CometExecSuite.scala | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 99007d0c91..d262e83f65 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -119,17 +119,19 @@ class CometExecSuite extends CometTestBase { } test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { - val table = "src" - withTable(table) { - withView("lv_noalias") { - sql(s"CREATE TABLE $table (key INT, value STRING) USING PARQUET") - sql(s"INSERT INTO $table VALUES(238, 'val_238')") + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { + val table = "src" + withTable(table) { + withView("lv_noalias") { + sql(s"CREATE TABLE $table (key INT, value STRING) USING PARQUET") + sql(s"INSERT INTO $table VALUES(238, 'val_238')") - sql( - "CREATE VIEW lv_noalias AS SELECT myTab.* FROM src " + - "LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab LIMIT 2") - val df = sql("SELECT * FROM lv_noalias a JOIN lv_noalias b ON a.key=b.key"); - checkSparkAnswer(df) + sql( + "CREATE VIEW lv_noalias AS SELECT myTab.* FROM src " + + "LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab LIMIT 2") + val df = sql("SELECT * FROM lv_noalias a JOIN lv_noalias b ON a.key=b.key"); + checkSparkAnswer(df) + } } } } From 10ef62ab49777fc5c8db11847745280614a6557c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 15 Nov 2024 10:16:41 -0700 Subject: [PATCH 05/47] Add changelog for 0.4.0 (#1089) --- dev/changelog/0.4.0.md | 108 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 108 insertions(+) create mode 100644 dev/changelog/0.4.0.md diff --git a/dev/changelog/0.4.0.md b/dev/changelog/0.4.0.md new file mode 100644 index 0000000000..69aa5c30b8 --- /dev/null +++ b/dev/changelog/0.4.0.md @@ -0,0 +1,108 @@ + + +# DataFusion Comet 0.4.0 Changelog + +This release consists of 51 commits from 10 contributors. See credits at the end of this changelog for more information. + +**Fixed bugs:** + +- fix: Use the number of rows from underlying arrays instead of logical row count from RecordBatch [#972](https://github.com/apache/datafusion-comet/pull/972) (viirya) +- fix: The spilled_bytes metric of CometSortExec should be size instead of time [#984](https://github.com/apache/datafusion-comet/pull/984) (Kontinuation) +- fix: Properly handle Java exceptions without error messages; fix loading of comet native library from java.library.path [#982](https://github.com/apache/datafusion-comet/pull/982) (Kontinuation) +- fix: Fallback to Spark if scan has meta columns [#997](https://github.com/apache/datafusion-comet/pull/997) (viirya) +- fix: Fallback to Spark if named_struct contains duplicate field names [#1016](https://github.com/apache/datafusion-comet/pull/1016) (viirya) +- fix: Make comet-git-info.properties optional [#1027](https://github.com/apache/datafusion-comet/pull/1027) (andygrove) +- fix: TopK operator should return correct results on dictionary column with nulls [#1033](https://github.com/apache/datafusion-comet/pull/1033) (viirya) +- fix: need default value for getSizeAsMb(EXECUTOR_MEMORY.key) [#1046](https://github.com/apache/datafusion-comet/pull/1046) (neyama) + +**Performance related:** + +- perf: Remove one redundant CopyExec for SMJ [#962](https://github.com/apache/datafusion-comet/pull/962) (andygrove) +- perf: Add experimental feature to replace SortMergeJoin with ShuffledHashJoin [#1007](https://github.com/apache/datafusion-comet/pull/1007) (andygrove) +- perf: Cache jstrings during metrics collection [#1029](https://github.com/apache/datafusion-comet/pull/1029) (mbutrovich) + +**Implemented enhancements:** + +- feat: Support `GetArrayStructFields` expression [#993](https://github.com/apache/datafusion-comet/pull/993) (Kimahriman) +- feat: Implement bloom_filter_agg [#987](https://github.com/apache/datafusion-comet/pull/987) (mbutrovich) +- feat: Support more types with BloomFilterAgg [#1039](https://github.com/apache/datafusion-comet/pull/1039) (mbutrovich) +- feat: Implement CAST from struct to string [#1066](https://github.com/apache/datafusion-comet/pull/1066) (andygrove) +- feat: Use official DataFusion 43 release [#1070](https://github.com/apache/datafusion-comet/pull/1070) (andygrove) +- feat: Implement CAST between struct types [#1074](https://github.com/apache/datafusion-comet/pull/1074) (andygrove) +- feat: support array_append [#1072](https://github.com/apache/datafusion-comet/pull/1072) (NoeB) +- feat: Require offHeap memory to be enabled (always use unified memory) [#1062](https://github.com/apache/datafusion-comet/pull/1062) (andygrove) + +**Documentation updates:** + +- doc: add documentation interlinks [#975](https://github.com/apache/datafusion-comet/pull/975) (comphead) +- docs: Add IntelliJ documentation for generated source code [#985](https://github.com/apache/datafusion-comet/pull/985) (mbutrovich) +- docs: Update tuning guide [#995](https://github.com/apache/datafusion-comet/pull/995) (andygrove) +- docs: Various documentation improvements [#1005](https://github.com/apache/datafusion-comet/pull/1005) (andygrove) +- docs: clarify that Maven central only has jars for Linux [#1009](https://github.com/apache/datafusion-comet/pull/1009) (andygrove) +- doc: fix K8s links and doc [#1058](https://github.com/apache/datafusion-comet/pull/1058) (comphead) +- docs: Update benchmarking.md [#1085](https://github.com/apache/datafusion-comet/pull/1085) (rluvaton-flarion) + +**Other:** + +- chore: Generate changelog for 0.3.0 release [#964](https://github.com/apache/datafusion-comet/pull/964) (andygrove) +- chore: fix publish-to-maven script [#966](https://github.com/apache/datafusion-comet/pull/966) (andygrove) +- chore: Update benchmarks results based on 0.3.0-rc1 [#969](https://github.com/apache/datafusion-comet/pull/969) (andygrove) +- chore: update rem expression guide [#976](https://github.com/apache/datafusion-comet/pull/976) (kazuyukitanimura) +- chore: Enable additional CreateArray tests [#928](https://github.com/apache/datafusion-comet/pull/928) (Kimahriman) +- chore: fix compatibility guide [#978](https://github.com/apache/datafusion-comet/pull/978) (kazuyukitanimura) +- chore: Update for 0.3.0 release, prepare for 0.4.0 development [#970](https://github.com/apache/datafusion-comet/pull/970) (andygrove) +- chore: Don't transform the HashAggregate to CometHashAggregate if Comet shuffle is disabled [#991](https://github.com/apache/datafusion-comet/pull/991) (viirya) +- chore: Make parquet reader options Comet options instead of Hadoop options [#968](https://github.com/apache/datafusion-comet/pull/968) (parthchandra) +- chore: remove legacy comet-spark-shell [#1013](https://github.com/apache/datafusion-comet/pull/1013) (andygrove) +- chore: Reserve memory for native shuffle writer per partition [#988](https://github.com/apache/datafusion-comet/pull/988) (viirya) +- chore: Bump arrow-rs to 53.1.0 and datafusion [#1001](https://github.com/apache/datafusion-comet/pull/1001) (kazuyukitanimura) +- chore: Revert "chore: Reserve memory for native shuffle writer per partition (#988)" [#1020](https://github.com/apache/datafusion-comet/pull/1020) (viirya) +- minor: Remove hard-coded version number from Dockerfile [#1025](https://github.com/apache/datafusion-comet/pull/1025) (andygrove) +- chore: Reserve memory for native shuffle writer per partition [#1022](https://github.com/apache/datafusion-comet/pull/1022) (viirya) +- chore: Improve error handling when native lib fails to load [#1000](https://github.com/apache/datafusion-comet/pull/1000) (andygrove) +- chore: Use twox-hash 2.0 xxhash64 oneshot api instead of custom implementation [#1041](https://github.com/apache/datafusion-comet/pull/1041) (NoeB) +- chore: Refactor Arrow Array and Schema allocation in ColumnReader and MetadataColumnReader [#1047](https://github.com/apache/datafusion-comet/pull/1047) (viirya) +- minor: Refactor binary expr serde to reduce code duplication [#1053](https://github.com/apache/datafusion-comet/pull/1053) (andygrove) +- chore: Upgrade to DataFusion 43.0.0-rc1 [#1057](https://github.com/apache/datafusion-comet/pull/1057) (andygrove) +- chore: Refactor UnaryExpr and MathExpr in protobuf [#1056](https://github.com/apache/datafusion-comet/pull/1056) (andygrove) +- minor: use defaults instead of hard-coding values [#1060](https://github.com/apache/datafusion-comet/pull/1060) (andygrove) +- minor: refactor UnaryExpr handling to make code more concise [#1065](https://github.com/apache/datafusion-comet/pull/1065) (andygrove) +- chore: Refactor binary and math expression serde code [#1069](https://github.com/apache/datafusion-comet/pull/1069) (andygrove) +- chore: Simplify CometShuffleMemoryAllocator to use Spark unified memory allocator [#1063](https://github.com/apache/datafusion-comet/pull/1063) (viirya) +- test: Restore one test in CometExecSuite by adding COMET_SHUFFLE_MODE config [#1087](https://github.com/apache/datafusion-comet/pull/1087) (viirya) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 19 Andy Grove + 13 Matt Butrovich + 8 Liang-Chi Hsieh + 3 KAZUYUKI TANIMURA + 2 Adam Binford + 2 Kristin Cowalcijk + 1 NoeB + 1 Oleks V + 1 Parth Chandra + 1 neyama +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. From 0c9a403ffaaee7247015edcd3472d2a6e86254dc Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 15 Nov 2024 11:56:56 -0700 Subject: [PATCH 06/47] chore: Prepare for 0.5.0 development (#1090) * Update version number for build * update docs --- .../actions/setup-spark-builder/action.yaml | 2 +- .github/workflows/spark_sql_test.yml | 2 +- .github/workflows/spark_sql_test_ansi.yml | 2 +- benchmarks/README.md | 2 +- common/pom.xml | 2 +- dev/diffs/3.4.3.diff | 2 +- dev/diffs/3.5.1.diff | 2 +- dev/diffs/4.0.0-preview1.diff | 2 +- docs/source/contributor-guide/debugging.md | 2 +- docs/source/user-guide/installation.md | 4 +- fuzz-testing/README.md | 6 +- fuzz-testing/pom.xml | 2 +- native/Cargo.lock | 70 +++++++++---------- native/Cargo.toml | 6 +- pom.xml | 2 +- spark-integration/pom.xml | 2 +- spark/pom.xml | 2 +- 17 files changed, 56 insertions(+), 56 deletions(-) diff --git a/.github/actions/setup-spark-builder/action.yaml b/.github/actions/setup-spark-builder/action.yaml index ebe8e0dc37..1bd37d6b17 100644 --- a/.github/actions/setup-spark-builder/action.yaml +++ b/.github/actions/setup-spark-builder/action.yaml @@ -29,7 +29,7 @@ inputs: comet-version: description: 'The Comet version to use for Spark' required: true - default: '0.4.0-SNAPSHOT' + default: '0.5.0-SNAPSHOT' runs: using: "composite" steps: diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 05c095b2f5..477e3a1ab9 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -71,7 +71,7 @@ jobs: with: spark-version: ${{ matrix.spark-version.full }} spark-short-version: ${{ matrix.spark-version.short }} - comet-version: '0.4.0-SNAPSHOT' # TODO: get this from pom.xml + comet-version: '0.5.0-SNAPSHOT' # TODO: get this from pom.xml - name: Run Spark tests run: | cd apache-spark diff --git a/.github/workflows/spark_sql_test_ansi.yml b/.github/workflows/spark_sql_test_ansi.yml index 06a5b2c8e7..e1d8388fb1 100644 --- a/.github/workflows/spark_sql_test_ansi.yml +++ b/.github/workflows/spark_sql_test_ansi.yml @@ -69,7 +69,7 @@ jobs: with: spark-version: ${{ matrix.spark-version.full }} spark-short-version: ${{ matrix.spark-version.short }} - comet-version: '0.4.0-SNAPSHOT' # TODO: get this from pom.xml + comet-version: '0.5.0-SNAPSHOT' # TODO: get this from pom.xml - name: Run Spark tests run: | cd apache-spark diff --git a/benchmarks/README.md b/benchmarks/README.md index 1042762d99..97877a3447 100644 --- a/benchmarks/README.md +++ b/benchmarks/README.md @@ -62,7 +62,7 @@ docker push localhost:32000/apache/datafusion-comet-tpcbench:latest export SPARK_MASTER=k8s://https://127.0.0.1:16443 export COMET_DOCKER_IMAGE=localhost:32000/apache/datafusion-comet-tpcbench:latest # Location of Comet JAR within the Docker image -export COMET_JAR=/opt/spark/jars/comet-spark-spark3.4_2.12-0.2.0-SNAPSHOT.jar +export COMET_JAR=/opt/spark/jars/comet-spark-spark3.4_2.12-0.5.0-SNAPSHOT.jar $SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ diff --git a/common/pom.xml b/common/pom.xml index cda873abbc..91109edf5d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.datafusion comet-parent-spark${spark.version.short}_${scala.binary.version} - 0.4.0-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 6ac55f95c9..12f739848c 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -7,7 +7,7 @@ index d3544881af1..bf0e2b53c70 100644 2.5.1 2.0.8 + 3.4 -+ 0.4.0-SNAPSHOT ++ 0.5.0-SNAPSHOT - -
Spark Executor
JVM Code
Comet Parquet Reader


IO and Decompression
Native Code
Native Execution Plan
Parquet Decoding
Shuffle Files
executePlan()
CometExecIterator
next()
Spark Execution Logic
decode()
next()
\ No newline at end of file diff --git a/docs/source/_static/images/CometOverviewDetailed.drawio b/docs/source/_static/images/CometOverviewDetailed.drawio deleted file mode 100644 index ff7f4c5911..0000000000 --- a/docs/source/_static/images/CometOverviewDetailed.drawio +++ /dev/null @@ -1,94 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/docs/source/_static/images/CometOverviewDetailed.drawio.svg b/docs/source/_static/images/CometOverviewDetailed.drawio.svg deleted file mode 100644 index 0f29083b11..0000000000 --- a/docs/source/_static/images/CometOverviewDetailed.drawio.svg +++ /dev/null @@ -1,4 +0,0 @@ - - - -
Spark Executor
Spark Driver
Spark Logical Plan
Spark Physical Plan
Comet Physical Plan
protobuf intermediate representation
Native Execution Plan
Comet Physical Plan
protobuf intermediate representation
Shuffle Files
\ No newline at end of file diff --git a/docs/source/_static/images/comet-dataflow.excalidraw b/docs/source/_static/images/comet-dataflow.excalidraw new file mode 100644 index 0000000000..dd12099834 --- /dev/null +++ b/docs/source/_static/images/comet-dataflow.excalidraw @@ -0,0 +1,2134 @@ +{ + "type": "excalidraw", + "version": 2, + "source": "https://excalidraw.com", + "elements": [ + { + "id": "dDrwaYB6MkVSDP_FHWS-F", + "type": "rectangle", + "x": 825.6666870117188, + "y": 116.83334350585938, + "width": 321.9999999999999, + "height": 324, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#ffd8a8", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "Zz", + "roundness": { + "type": 3 + }, + "seed": 1163356465, + "version": 243, + "versionNonce": 743550265, + "isDeleted": false, + "boundElements": [ + { + "id": "u84B3vp5oTVNXI5uXsZ-r", + "type": "arrow" + }, + { + "id": "dlyj3Gno71fx16oqbbjXF", + "type": "arrow" + } + ], + "updated": 1733167126280, + "link": null, + "locked": false + }, + { + "id": "8pVcXTnP3tefe_O3kTE0b", + "type": "text", + "x": 467.66668701171875, + "y": 48.833343505859375, + "width": 61, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "dotted", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "aS", + "roundness": null, + "seed": 306458015, + "version": 181, + "versionNonce": 110788633, + "isDeleted": false, + "boundElements": [], + "updated": 1733167570417, + "link": null, + "locked": false, + "text": "JVM", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "JVM", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "Ax7J0LoYh5TwQoRBM47cz", + "type": "text", + "x": 941.6666870117188, + "y": 56.833343505859375, + "width": 97, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "dotted", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "aT", + "roundness": null, + "seed": 1762016049, + "version": 173, + "versionNonce": 1117284823, + "isDeleted": false, + "boundElements": [], + "updated": 1733167564367, + "link": null, + "locked": false, + "text": "Native", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "Native", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "lSUrwgLq2W49ULouPfm0h", + "type": "rectangle", + "x": 868.1666870117188, + "y": 168.83334350585938, + "width": 245.00000000000006, + "height": 83.99999999999997, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "aU", + "roundness": { + "type": 3 + }, + "seed": 1188956881, + "version": 337, + "versionNonce": 502265527, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "M6A-P7jOlvlDx-Kc0nsrQ" + }, + { + "id": "MqWIMNh5n51EVvWedfTIA", + "type": "arrow" + }, + { + "id": "GPIY241P4rRnRn48VdbYe", + "type": "arrow" + }, + { + "id": "6KmKXuc4aon2_yKt2fdZE", + "type": "arrow" + }, + { + "id": "ou2srC_Up4kjWcmgzdEH4", + "type": "arrow" + } + ], + "updated": 1733167585167, + "link": null, + "locked": false + }, + { + "id": "M6A-P7jOlvlDx-Kc0nsrQ", + "type": "text", + "x": 903.4791870117188, + "y": 198.33334350585938, + "width": 174.375, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "aV", + "roundness": null, + "seed": 1968605361, + "version": 333, + "versionNonce": 1113091385, + "isDeleted": false, + "boundElements": [], + "updated": 1733166946960, + "link": null, + "locked": false, + "text": "ShuffleWriterExec", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "lSUrwgLq2W49ULouPfm0h", + "originalText": "ShuffleWriterExec", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "GHKyE6o_at1-J0KO1mWpt", + "type": "rectangle", + "x": 363.85928382109046, + "y": 505.8341459769945, + "width": 262.9999999999998, + "height": 93.99611799705886, + "angle": 0.003703686768755432, + "strokeColor": "#1e1e1e", + "backgroundColor": "#a5d8ff", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "aa", + "roundness": { + "type": 3 + }, + "seed": 952999857, + "version": 632, + "versionNonce": 906119703, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "yHFb7s7QYOWZst8xXlFG2" + }, + { + "id": "Jd5Fqfx6eFl_OJ6x0TUki", + "type": "arrow" + }, + { + "id": "7KEns52XY_jok50o5G5op", + "type": "arrow" + }, + { + "id": "quv5xELoqOR6W5SJipUrY", + "type": "arrow" + }, + { + "id": "kQzva6A57whXeUyhhNxOl", + "type": "arrow" + }, + { + "id": "Pjo3gnqBVibIixMHpFvkK", + "type": "arrow" + } + ], + "updated": 1733167372550, + "link": null, + "locked": false + }, + { + "id": "yHFb7s7QYOWZst8xXlFG2", + "type": "text", + "x": 399.93428382109033, + "y": 540.3322049755238, + "width": 190.85000000000002, + "height": 25, + "angle": 0.003703686768755432, + "strokeColor": "#1e1e1e", + "backgroundColor": "#a5d8ff", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "ab", + "roundness": null, + "seed": 1354040959, + "version": 598, + "versionNonce": 226422583, + "isDeleted": false, + "boundElements": [], + "updated": 1733167372550, + "link": null, + "locked": false, + "text": "CometExecIterator", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "GHKyE6o_at1-J0KO1mWpt", + "originalText": "CometExecIterator", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "Iv4y4lEpq-EOkw5bBAWNA", + "type": "text", + "x": 930.6666870117188, + "y": 130.83334350585938, + "width": 109.9000015258789, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#ffd8a8", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b06", + "roundness": null, + "seed": 952057055, + "version": 92, + "versionNonce": 52977177, + "isDeleted": false, + "boundElements": [], + "updated": 1733166946960, + "link": null, + "locked": false, + "text": "Native Plan", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "Native Plan", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "Ro2R78aPw-luRF_bB2EKU", + "type": "rectangle", + "x": 366.34678047371074, + "y": 307.83595662631933, + "width": 262.99999999999983, + "height": 92.00353907094141, + "angle": 0.003703686768755432, + "strokeColor": "#1e1e1e", + "backgroundColor": "#a5d8ff", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0g", + "roundness": { + "type": 3 + }, + "seed": 959895479, + "version": 644, + "versionNonce": 1083149527, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "twg3z-vK6jWmVl4xySGde" + }, + { + "id": "u84B3vp5oTVNXI5uXsZ-r", + "type": "arrow" + }, + { + "id": "IISSP3sEmCbjsvI4SFgaX", + "type": "arrow" + }, + { + "id": "7KEns52XY_jok50o5G5op", + "type": "arrow" + }, + { + "id": "quv5xELoqOR6W5SJipUrY", + "type": "arrow" + } + ], + "updated": 1733167372551, + "link": null, + "locked": false + }, + { + "id": "twg3z-vK6jWmVl4xySGde", + "type": "text", + "x": 396.18428047371066, + "y": 341.33772616179004, + "width": 203.32500000000002, + "height": 25, + "angle": 0.003703686768755432, + "strokeColor": "#1e1e1e", + "backgroundColor": "#a5d8ff", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0h", + "roundness": null, + "seed": 34654423, + "version": 631, + "versionNonce": 1121311223, + "isDeleted": false, + "boundElements": [], + "updated": 1733167372551, + "link": null, + "locked": false, + "text": "CometBatchIterator", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "Ro2R78aPw-luRF_bB2EKU", + "originalText": "CometBatchIterator", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "macb6DKtgx8DhcqjKk6no", + "type": "rectangle", + "x": 366.1634633724364, + "y": 157.33528450732996, + "width": 262.9999999999998, + "height": 93.99611799705886, + "angle": 0.003703686768755432, + "strokeColor": "#1e1e1e", + "backgroundColor": "#a5d8ff", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0i", + "roundness": { + "type": 3 + }, + "seed": 1827361271, + "version": 674, + "versionNonce": 1149488599, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "HzfSeR8C3p6yYRHlGGIdM" + }, + { + "id": "dlyj3Gno71fx16oqbbjXF", + "type": "arrow" + }, + { + "id": "MqWIMNh5n51EVvWedfTIA", + "type": "arrow" + } + ], + "updated": 1733167385065, + "link": null, + "locked": false + }, + { + "id": "HzfSeR8C3p6yYRHlGGIdM", + "type": "text", + "x": 402.2384633724363, + "y": 191.83334350585938, + "width": 190.85000000000002, + "height": 25, + "angle": 0.003703686768755432, + "strokeColor": "#1e1e1e", + "backgroundColor": "#a5d8ff", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0j", + "roundness": null, + "seed": 901511959, + "version": 643, + "versionNonce": 1747825847, + "isDeleted": false, + "boundElements": [], + "updated": 1733167372551, + "link": null, + "locked": false, + "text": "CometExecIterator", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "macb6DKtgx8DhcqjKk6no", + "originalText": "CometExecIterator", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "7VTYHzsqQvUuKMy0ShKZn", + "type": "rectangle", + "x": 871.1634633724364, + "y": 304.3333435058594, + "width": 245.00000000000006, + "height": 83.99999999999997, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0k", + "roundness": { + "type": 3 + }, + "seed": 1785572407, + "version": 379, + "versionNonce": 1216788985, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "YKETugAZBRtG7oeas8CPz" + }, + { + "id": "IISSP3sEmCbjsvI4SFgaX", + "type": "arrow" + }, + { + "id": "u84B3vp5oTVNXI5uXsZ-r", + "type": "arrow" + }, + { + "id": "GPIY241P4rRnRn48VdbYe", + "type": "arrow" + }, + { + "id": "6KmKXuc4aon2_yKt2fdZE", + "type": "arrow" + } + ], + "updated": 1733167417649, + "link": null, + "locked": false + }, + { + "id": "YKETugAZBRtG7oeas8CPz", + "type": "text", + "x": 947.8009641353758, + "y": 333.8333435058594, + "width": 91.7249984741211, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0l", + "roundness": null, + "seed": 2121862487, + "version": 357, + "versionNonce": 1828219865, + "isDeleted": false, + "boundElements": [], + "updated": 1733166946960, + "link": null, + "locked": false, + "text": "ScanExec", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "7VTYHzsqQvUuKMy0ShKZn", + "originalText": "ScanExec", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "_a8_bfztXdYiD4AXJxPee", + "type": "rectangle", + "x": 820.6634633724364, + "y": 473.3333435058594, + "width": 334.9999999999999, + "height": 329, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#ffd8a8", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0m", + "roundness": { + "type": 3 + }, + "seed": 1577231703, + "version": 409, + "versionNonce": 1832634263, + "isDeleted": false, + "boundElements": [ + { + "id": "kQzva6A57whXeUyhhNxOl", + "type": "arrow" + }, + { + "id": "Pjo3gnqBVibIixMHpFvkK", + "type": "arrow" + } + ], + "updated": 1733167465343, + "link": null, + "locked": false + }, + { + "id": "aiAipugp154jY5IgHqjTm", + "type": "rectangle", + "x": 862.1634633724364, + "y": 535.3333435058594, + "width": 245.00000000000006, + "height": 83.99999999999997, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0n", + "roundness": { + "type": 3 + }, + "seed": 1666310775, + "version": 392, + "versionNonce": 1164820153, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "XPpjuVi7ZYpwo3X03G9P1" + }, + { + "id": "Pjo3gnqBVibIixMHpFvkK", + "type": "arrow" + } + ], + "updated": 1733167345582, + "link": null, + "locked": false + }, + { + "id": "XPpjuVi7ZYpwo3X03G9P1", + "type": "text", + "x": 924.6759633724364, + "y": 564.8333435058594, + "width": 119.97500000000001, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0o", + "roundness": null, + "seed": 838630295, + "version": 403, + "versionNonce": 1508263831, + "isDeleted": false, + "boundElements": [], + "updated": 1733166982872, + "link": null, + "locked": false, + "text": "ProjectExec", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "aiAipugp154jY5IgHqjTm", + "originalText": "ProjectExec", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "aE2QBjfmrpzBTUB_t6QRY", + "type": "text", + "x": 924.6634633724364, + "y": 497.3333435058594, + "width": 109.9000015258789, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#ffd8a8", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0p", + "roundness": null, + "seed": 1043787959, + "version": 150, + "versionNonce": 1187544183, + "isDeleted": false, + "boundElements": [], + "updated": 1733166977658, + "link": null, + "locked": false, + "text": "Native Plan", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "Native Plan", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "vG9tTZvROj2ybA4oAS_bb", + "type": "rectangle", + "x": 864.160239733154, + "y": 671.8333435058594, + "width": 245.00000000000006, + "height": 83.99999999999997, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0q", + "roundness": { + "type": 3 + }, + "seed": 27640279, + "version": 529, + "versionNonce": 1105701913, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "oWkZozTacCvv40wRG7g3s" + } + ], + "updated": 1733167462816, + "link": null, + "locked": false + }, + { + "id": "oWkZozTacCvv40wRG7g3s", + "type": "text", + "x": 978.160239733154, + "y": 701.3333435058594, + "width": 17, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0r", + "roundness": null, + "seed": 1297968887, + "version": 526, + "versionNonce": 271368441, + "isDeleted": false, + "boundElements": [], + "updated": 1733167462816, + "link": null, + "locked": false, + "text": "...", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "vG9tTZvROj2ybA4oAS_bb", + "originalText": "...", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "kQzva6A57whXeUyhhNxOl", + "type": "arrow", + "x": 627.9453883765393, + "y": 529.8293445331748, + "width": 192.71807499589704, + "height": 0.234522891430629, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0w", + "roundness": { + "type": 2 + }, + "seed": 562773463, + "version": 217, + "versionNonce": 611157943, + "isDeleted": false, + "boundElements": [], + "updated": 1733167475920, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 192.71807499589704, + 0.234522891430629 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "GHKyE6o_at1-J0KO1mWpt", + "focus": -0.48947127224675757, + "gap": 1, + "fixedPoint": null + }, + "endBinding": { + "elementId": "C3-eUJazhRorbXp9Um-Mo", + "focus": -1.9941089907787155, + "gap": 12.73052391874603, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "Pjo3gnqBVibIixMHpFvkK", + "type": "arrow", + "x": 861.6634633724364, + "y": 571.3333435058594, + "width": 233.87028528948713, + "height": 0.4072197033743805, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0y", + "roundness": { + "type": 2 + }, + "seed": 189975865, + "version": 190, + "versionNonce": 1899895735, + "isDeleted": false, + "boundElements": [], + "updated": 1733167372551, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + -233.87028528948713, + -0.4072197033743805 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "aiAipugp154jY5IgHqjTm", + "focus": 0.1262072643242283, + "gap": 1, + "fixedPoint": null + }, + "endBinding": { + "elementId": "GHKyE6o_at1-J0KO1mWpt", + "focus": 0.37801089214584216, + "gap": 1, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "u84B3vp5oTVNXI5uXsZ-r", + "type": "arrow", + "x": 867.6634633724364, + "y": 335.3333435058594, + "width": 235.9983810536769, + "height": 0.5628844927418868, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b0z", + "roundness": { + "type": 2 + }, + "seed": 849585047, + "version": 139, + "versionNonce": 2098561815, + "isDeleted": false, + "boundElements": null, + "updated": 1733167372551, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + -235.9983810536769, + 0.5628844927418868 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "7VTYHzsqQvUuKMy0ShKZn", + "focus": 0.261904761904762, + "gap": 3.5, + "fixedPoint": null + }, + "endBinding": { + "elementId": "Ro2R78aPw-luRF_bB2EKU", + "focus": -0.3765315568105985, + "gap": 2.2509344960505473, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "IISSP3sEmCbjsvI4SFgaX", + "type": "arrow", + "x": 630.6644917368169, + "y": 368.0556851230956, + "width": 238.99897163561945, + "height": 0.7223416172362249, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b12", + "roundness": { + "type": 2 + }, + "seed": 1980422201, + "version": 131, + "versionNonce": 1606617143, + "isDeleted": false, + "boundElements": null, + "updated": 1733167372551, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 238.99897163561945, + -0.7223416172362249 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "Ro2R78aPw-luRF_bB2EKU", + "focus": 0.31181090317651905, + "gap": 1.3694590603334404, + "fixedPoint": null + }, + "endBinding": { + "elementId": "7VTYHzsqQvUuKMy0ShKZn", + "focus": -0.5000000000000002, + "gap": 1.5, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "7KEns52XY_jok50o5G5op", + "type": "arrow", + "x": 437.66346337243635, + "y": 399.3333435058594, + "width": 4, + "height": 104, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b13", + "roundness": { + "type": 2 + }, + "seed": 1651841465, + "version": 252, + "versionNonce": 1005623161, + "isDeleted": false, + "boundElements": null, + "updated": 1733167373032, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 4, + 104 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "Ro2R78aPw-luRF_bB2EKU", + "focus": 0.46419678699387723, + "gap": 1, + "fixedPoint": null + }, + "endBinding": { + "elementId": "GHKyE6o_at1-J0KO1mWpt", + "focus": -0.3880655447790852, + "gap": 2.3015909311958467, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "quv5xELoqOR6W5SJipUrY", + "type": "arrow", + "x": 555.6634633724364, + "y": 503.3333435058594, + "width": 0, + "height": 103, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b14", + "roundness": { + "type": 2 + }, + "seed": 1677615735, + "version": 236, + "versionNonce": 55168313, + "isDeleted": false, + "boundElements": null, + "updated": 1733167373032, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 0, + -103 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "GHKyE6o_at1-J0KO1mWpt", + "focus": 0.4579838276843583, + "gap": 2.723810257547825, + "fixedPoint": null + }, + "endBinding": { + "elementId": "Ro2R78aPw-luRF_bB2EKU", + "focus": -0.43910468547182224, + "gap": 1, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "dlyj3Gno71fx16oqbbjXF", + "type": "arrow", + "x": 632.6636522386541, + "y": 181.2823496270506, + "width": 193.99981113378226, + "height": 0.9490061211912177, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b15", + "roundness": { + "type": 2 + }, + "seed": 1072669367, + "version": 132, + "versionNonce": 1271110743, + "isDeleted": false, + "boundElements": null, + "updated": 1733167393399, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 193.99981113378226, + -0.9490061211912177 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "macb6DKtgx8DhcqjKk6no", + "focus": -0.4652381310069499, + "gap": 3.4138894826694752, + "fixedPoint": null + }, + "endBinding": { + "elementId": "OFwuou30qsm3aMZ96ASUO", + "focus": -1.5667144994299218, + "gap": 7.5, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "C3-eUJazhRorbXp9Um-Mo", + "type": "text", + "x": 664.6634633724364, + "y": 492.3333435058594, + "width": 189, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b16", + "roundness": null, + "seed": 755500537, + "version": 76, + "versionNonce": 1747049559, + "isDeleted": false, + "boundElements": [ + { + "id": "kQzva6A57whXeUyhhNxOl", + "type": "arrow" + } + ], + "updated": 1733167475516, + "link": null, + "locked": false, + "text": "executePlan()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "executePlan()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "OFwuou30qsm3aMZ96ASUO", + "type": "text", + "x": 669.1634633724364, + "y": 147.83334350585938, + "width": 189, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b17", + "roundness": null, + "seed": 1806263479, + "version": 124, + "versionNonce": 1646888249, + "isDeleted": false, + "boundElements": [ + { + "id": "dlyj3Gno71fx16oqbbjXF", + "type": "arrow" + } + ], + "updated": 1733167389568, + "link": null, + "locked": false, + "text": "executePlan()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "executePlan()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "2ZoBSXI-amAjEfzxoQ17b", + "type": "text", + "x": 749.1634633724364, + "y": 308.8333435058594, + "width": 111, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b18", + "roundness": null, + "seed": 1238305721, + "version": 97, + "versionNonce": 1368434199, + "isDeleted": false, + "boundElements": [], + "updated": 1733167188224, + "link": null, + "locked": false, + "text": "next()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "next()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "1-lAVH11BDSwJVoMYl80T", + "type": "text", + "x": 371.16346337243635, + "y": 439.8333435058594, + "width": 111, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b19", + "roundness": null, + "seed": 756108375, + "version": 211, + "versionNonce": 161358135, + "isDeleted": false, + "boundElements": [], + "updated": 1733167372551, + "link": null, + "locked": false, + "text": "next()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "next()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "VFm7kotI1oNa1rIxLMh6W", + "type": "text", + "x": 676.6634633724364, + "y": 376.3333435058594, + "width": 147, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1A", + "roundness": null, + "seed": 1623222905, + "version": 76, + "versionNonce": 1030050969, + "isDeleted": false, + "boundElements": null, + "updated": 1733167271120, + "link": null, + "locked": false, + "text": "exportBatch()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "exportBatch()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "LZtRekUnAEPkjsECzd7zb", + "type": "text", + "x": 663.6634633724364, + "y": 575.3333435058594, + "width": 217, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1B", + "roundness": null, + "seed": 187512855, + "version": 127, + "versionNonce": 1917573399, + "isDeleted": false, + "boundElements": null, + "updated": 1733167377483, + "link": null, + "locked": false, + "text": "importVectors()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "importVectors()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "MqWIMNh5n51EVvWedfTIA", + "type": "arrow", + "x": 868.6634633724364, + "y": 220.33334350585938, + "width": 239, + "height": 1, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1C", + "roundness": { + "type": 2 + }, + "seed": 685490007, + "version": 35, + "versionNonce": 652639415, + "isDeleted": false, + "boundElements": null, + "updated": 1733167385065, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + -239, + 1 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "lSUrwgLq2W49ULouPfm0h", + "focus": -0.2114558118557865, + "gap": 1, + "fixedPoint": null + }, + "endBinding": { + "elementId": "macb6DKtgx8DhcqjKk6no", + "focus": 0.3654122251883526, + "gap": 1, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "RYdCN0xyvNHqlA1WDARNx", + "type": "text", + "x": 670.1634633724364, + "y": 225.83334350585938, + "width": 217, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1D", + "roundness": null, + "seed": 546285145, + "version": 165, + "versionNonce": 1950128183, + "isDeleted": false, + "boundElements": [], + "updated": 1733167400420, + "link": null, + "locked": false, + "text": "importVectors()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "importVectors()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "GPIY241P4rRnRn48VdbYe", + "type": "arrow", + "x": 941.6634633724364, + "y": 264.3333435058594, + "width": 2, + "height": 32, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1E", + "roundness": { + "type": 2 + }, + "seed": 1958688375, + "version": 17, + "versionNonce": 1070980535, + "isDeleted": false, + "boundElements": null, + "updated": 1733167413149, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 2, + 32 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "lSUrwgLq2W49ULouPfm0h", + "focus": 0.4183574316825765, + "gap": 11.500000000000014, + "fixedPoint": null + }, + "endBinding": { + "elementId": "7VTYHzsqQvUuKMy0ShKZn", + "focus": -0.37462537462537454, + "gap": 8, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "6KmKXuc4aon2_yKt2fdZE", + "type": "arrow", + "x": 1052.6634633724364, + "y": 290.3333435058594, + "width": 1, + "height": 30, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1F", + "roundness": { + "type": 2 + }, + "seed": 1971585785, + "version": 15, + "versionNonce": 1869550297, + "isDeleted": false, + "boundElements": null, + "updated": 1733167417649, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + -1, + -30 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "7VTYHzsqQvUuKMy0ShKZn", + "focus": 0.4912563895614742, + "gap": 14, + "fixedPoint": null + }, + "endBinding": { + "elementId": "lSUrwgLq2W49ULouPfm0h", + "focus": -0.4789893168742339, + "gap": 7.500000000000014, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "0HMrFdchM3CqZB7BeDX-8", + "type": "text", + "x": 866.1634633724364, + "y": 269.8333435058594, + "width": 111, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1G", + "roundness": null, + "seed": 752246361, + "version": 154, + "versionNonce": 350456215, + "isDeleted": false, + "boundElements": [], + "updated": 1733167430100, + "link": null, + "locked": false, + "text": "next()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "next()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "laDFG84hR_vbCVHWMp16w", + "type": "arrow", + "x": 942.7269220990613, + "y": 628.5394176529115, + "width": 2, + "height": 32, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1H", + "roundness": { + "type": 2 + }, + "seed": 1602854839, + "version": 100, + "versionNonce": 1479430905, + "isDeleted": false, + "boundElements": [], + "updated": 1733167459736, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 2, + 32 + ] + ], + "lastCommittedPoint": null, + "startBinding": null, + "endBinding": null, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "PKnVxmYbX4yxvUMYxiED6", + "type": "arrow", + "x": 1053.7269220990613, + "y": 654.5394176529115, + "width": 1, + "height": 30, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1I", + "roundness": { + "type": 2 + }, + "seed": 1657442519, + "version": 98, + "versionNonce": 951815129, + "isDeleted": false, + "boundElements": [], + "updated": 1733167459736, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + -1, + -30 + ] + ], + "lastCommittedPoint": null, + "startBinding": null, + "endBinding": null, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + }, + { + "id": "3s_jHVea7P3zpwyhsYqNO", + "type": "text", + "x": 867.2269220990613, + "y": 634.0394176529115, + "width": 111, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1J", + "roundness": null, + "seed": 1863343607, + "version": 237, + "versionNonce": 149858489, + "isDeleted": false, + "boundElements": [], + "updated": 1733167459736, + "link": null, + "locked": false, + "text": "next()", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "next()", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "9t3RnkE-nqccuTgRaqj2w", + "type": "text", + "x": 492.66346337243635, + "y": 444.3333435058594, + "width": 60.000000000000014, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1K", + "roundness": null, + "seed": 1235248153, + "version": 108, + "versionNonce": 91341817, + "isDeleted": false, + "boundElements": null, + "updated": 1733167501667, + "link": null, + "locked": false, + "text": "batch", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "batch", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "3xNt8fnnjY9QtNoev_FMj", + "type": "text", + "x": 1070.6634633724364, + "y": 265.8333435058594, + "width": 60.000000000000014, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1L", + "roundness": null, + "seed": 1471691417, + "version": 154, + "versionNonce": 195092727, + "isDeleted": false, + "boundElements": [], + "updated": 1733167506301, + "link": null, + "locked": false, + "text": "batch", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "batch", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "p9iXL4J6GojVzXDDx1NVM", + "type": "text", + "x": 1071.6634633724364, + "y": 629.8333435058594, + "width": 60.000000000000014, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1M", + "roundness": null, + "seed": 951602999, + "version": 150, + "versionNonce": 595097273, + "isDeleted": false, + "boundElements": [], + "updated": 1733167509787, + "link": null, + "locked": false, + "text": "batch", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "left", + "verticalAlign": "top", + "containerId": null, + "originalText": "batch", + "autoResize": false, + "lineHeight": 1.25 + }, + { + "id": "xdQ0w3-b5BGEpSvQ2Uc8A", + "type": "rectangle", + "x": 1212.6634633724364, + "y": 131.83334350585938, + "width": 260, + "height": 128.99999999999997, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#eaddd7", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1N", + "roundness": { + "type": 3 + }, + "seed": 407784057, + "version": 373, + "versionNonce": 1796011255, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "h6I3dPlbn7bb50l-R-ZrT" + }, + { + "id": "ou2srC_Up4kjWcmgzdEH4", + "type": "arrow" + } + ], + "updated": 1733167585167, + "link": null, + "locked": false + }, + { + "id": "h6I3dPlbn7bb50l-R-ZrT", + "type": "text", + "x": 1282.2634633724363, + "y": 136.83334350585938, + "width": 120.80000000000001, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#eaddd7", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1O", + "roundness": null, + "seed": 1894814553, + "version": 326, + "versionNonce": 1273706233, + "isDeleted": false, + "boundElements": [], + "updated": 1733167557133, + "link": null, + "locked": false, + "text": "Shuffle Files", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "top", + "containerId": "xdQ0w3-b5BGEpSvQ2Uc8A", + "originalText": "Shuffle Files", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "rpX5p5xVzE-agyW9ssfpT", + "type": "rectangle", + "x": 1253.6634633724364, + "y": 182.33334350585938, + "width": 190, + "height": 45, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#ffec99", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1P", + "roundness": { + "type": 3 + }, + "seed": 337035321, + "version": 422, + "versionNonce": 1799090137, + "isDeleted": false, + "boundElements": [ + { + "type": "text", + "id": "ZHswgvVioPRH-MY0pzRZO" + } + ], + "updated": 1733167557133, + "link": null, + "locked": false + }, + { + "id": "ZHswgvVioPRH-MY0pzRZO", + "type": "text", + "x": 1264.0634648983153, + "y": 192.33334350585938, + "width": 169.1999969482422, + "height": 25, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#ffec99", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1Q", + "roundness": null, + "seed": 1336168729, + "version": 390, + "versionNonce": 1438813369, + "isDeleted": false, + "boundElements": [], + "updated": 1733167557133, + "link": null, + "locked": false, + "text": "Arrow IPC Batch", + "fontSize": 20, + "fontFamily": 5, + "textAlign": "center", + "verticalAlign": "middle", + "containerId": "rpX5p5xVzE-agyW9ssfpT", + "originalText": "Arrow IPC Batch", + "autoResize": true, + "lineHeight": 1.25 + }, + { + "id": "ou2srC_Up4kjWcmgzdEH4", + "type": "arrow", + "x": 1114.6634633724364, + "y": 210.33334350585938, + "width": 97, + "height": 2, + "angle": 0, + "strokeColor": "#1e1e1e", + "backgroundColor": "#b2f2bb", + "fillStyle": "solid", + "strokeWidth": 2, + "strokeStyle": "solid", + "roughness": 1, + "opacity": 100, + "groupIds": [], + "frameId": null, + "index": "b1R", + "roundness": { + "type": 2 + }, + "seed": 1875512793, + "version": 40, + "versionNonce": 1347291095, + "isDeleted": false, + "boundElements": null, + "updated": 1733167585167, + "link": null, + "locked": false, + "points": [ + [ + 0, + 0 + ], + [ + 97, + -2 + ] + ], + "lastCommittedPoint": null, + "startBinding": { + "elementId": "lSUrwgLq2W49ULouPfm0h", + "focus": 0.04618975520292551, + "gap": 1.496776360717604, + "fixedPoint": null + }, + "endBinding": { + "elementId": "xdQ0w3-b5BGEpSvQ2Uc8A", + "focus": -0.13841786234942072, + "gap": 1, + "fixedPoint": null + }, + "startArrowhead": null, + "endArrowhead": "arrow", + "elbowed": false + } + ], + "appState": { + "gridSize": 20, + "gridStep": 5, + "gridModeEnabled": false, + "viewBackgroundColor": "#ffffff" + }, + "files": {} +} \ No newline at end of file diff --git a/docs/source/_static/images/comet-dataflow.svg b/docs/source/_static/images/comet-dataflow.svg new file mode 100644 index 0000000000..20a573c1fa --- /dev/null +++ b/docs/source/_static/images/comet-dataflow.svg @@ -0,0 +1,10 @@ + + + + + + + + JVMNativeShuffleWriterExecCometExecIteratorNative PlanCometBatchIteratorCometExecIteratorScanExecProjectExecNative Plan...executePlan()executePlan()next()next()exportBatch()importVectors()importVectors()next()next()batchbatchbatchShuffle FilesArrow IPC Batch \ No newline at end of file diff --git a/docs/source/contributor-guide/plugin_overview.md b/docs/source/contributor-guide/plugin_overview.md index a211ca6b55..3e7c24f55a 100644 --- a/docs/source/contributor-guide/plugin_overview.md +++ b/docs/source/contributor-guide/plugin_overview.md @@ -28,11 +28,21 @@ following setting to the Spark configuration when launching `spark-shell` or `sp --conf spark.plugins=org.apache.spark.CometPlugin ``` +This class is loaded by Spark's plugin framework. It will be instantiated in the Spark driver only. Comet does not +provide any executor plugins. + +The plugin will update the current `SparkConf` with the extra configuration provided by Comet, such as executor memory +configuration. + +The plugin also registers `CometSparkSessionExtensions` with Spark's extension API. + +## CometSparkSessionExtensions + On initialization, this class registers two physical plan optimization rules with Spark: `CometScanRule` and `CometExecRule`. These rules run whenever a query stage is being planned during Adaptive Query Execution, and run once for the entire plan when Adaptive Query Execution is disabled. -## CometScanRule +### CometScanRule `CometScanRule` replaces any Parquet scans with Comet operators. There are different paths for Spark v1 and v2 data sources. @@ -43,13 +53,13 @@ Parquet data source but leverages native code for decoding Parquet row groups di Comet only supports a subset of data types and will fall back to Spark's scan if unsupported types exist. Comet can still accelerate the rest of the query execution in this case because `CometSparkToColumnarExec` will -convert the output from Spark's can to Arrow arrays. Note that both `spark.comet.exec.enabled=true` and +convert the output from Spark's scan to Arrow arrays. Note that both `spark.comet.exec.enabled=true` and `spark.comet.convert.parquet.enabled=true` must be set to enable this conversion. Refer to the [Supported Spark Data Types](https://datafusion.apache.org/comet/user-guide/datatypes.html) section in the contributor guide to see a list of currently supported data types. -## CometExecRule +### CometExecRule This rule traverses bottom-up from the original Spark plan and attempts to replace each operator with a Comet equivalent. For example, a `ProjectExec` will be replaced by `CometProjectExec`. @@ -64,25 +74,52 @@ of this could outweigh the benefits of running parts of the query stage natively ## Query Execution -Once the plan has been transformed, any consecutive Comet operators are combined into a `CometNativeExec` which contains -a serialized version of the plan (the serialization code can be found in `QueryPlanSerde`). When this operator is -executed, the serialized plan is passed to the native code when calling `Native.createPlan`. +Once the plan has been transformed, any consecutive native Comet operators are combined into a `CometNativeExec` which contains +a protocol buffer serialized version of the plan (the serialization code can be found in `QueryPlanSerde`). -In the native code there is a `PhysicalPlanner` struct (in `planner.rs`) which converts the serialized plan into an +Spark serializes the physical plan and sends it to the executors when executing tasks. The executors deserialize the +plan and invoke it. + +When `CometNativeExec` is invoked, it will pass the serialized protobuf plan into +`Native.createPlan`, which invokes the native code via JNI, where the plan is then deserialized. + +In the native code there is a `PhysicalPlanner` struct (in `planner.rs`) which converts the deserialized plan into an Apache DataFusion `ExecutionPlan`. In some cases, Comet provides specialized physical operators and expressions to override the DataFusion versions to ensure compatibility with Apache Spark. -`CometExecIterator` will invoke `Native.executePlan` to pull the next batch from the native plan. This is repeated -until no more batches are available (meaning that all data has been processed by the native plan). +The leaf nodes in the physical plan are always `ScanExec` and each of these operators will make a JNI call to +`CometBatchIterator.next()` to fetch the next input batch. The input could be a Comet native Parquet scan, +a Spark exchange, or another native plan. + +`CometNativeExec` creates a `CometExecIterator` and applies this iterator to the input RDD +partitions. Each call to `CometExecIterator.next()` will invoke `Native.executePlan`. Once the plan finishes +executing, the resulting Arrow batches are imported into the JVM using Arrow FFI. + +## Arrow + +Due to the hybrid execution model, it is necessary to pass batches of data between the JVM and native code. + +Comet uses a combination of Arrow FFI and Arrow IPC to achieve this. + +### Arrow FFI + +The foundation for Arrow FFI is the [Arrow C Data Interface], which provides a stable ABI-compatible interface for +accessing Arrow data structures from multiple languages. + +[Arrow C Data Interface]: https://arrow.apache.org/docs/format/CDataInterface.html + +- `CometExecIterator` invokes native plans and uses Arrow FFI to read the output batches +- Native `ScanExec` operators call `CometBatchIterator` via JNI to fetch input batches from the JVM + +### Arrow IPC -The leaf nodes in the physical plan are always `ScanExec` and these operators consume batches of Arrow data that were -prepared before the plan is executed. When `CometExecIterator` invokes `Native.executePlan` it passes the memory -addresses of these Arrow arrays to the native code. +Comet native shuffle uses Arrow IPC to write batches to the shuffle files. -![Diagram of Comet Native Execution](../../_static/images/CometOverviewDetailed.drawio.svg) +- `CometShuffleWriteProcessor` invokes a native plan to fetch batches and then passes them to native `ShuffleWriterExec` +- `CometBlockStoreShuffleReader` reads batches from shuffle files ## End to End Flow -The following diagram shows the end-to-end flow. +The following diagram shows an example of the end-to-end flow for a query stage. -![Diagram of Comet Native Parquet Scan](../../_static/images/CometNativeParquetReader.drawio.svg) +![Diagram of Comet Data Flow](../../_static/images/comet-dataflow.svg) From 2671e0cafbd6fe96c9af852ad7f34592e299a62a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 3 Dec 2024 17:40:54 -0700 Subject: [PATCH 22/47] Stop passing Java config map into native createPlan (#1101) --- native/core/src/execution/jni_api.rs | 69 +++++-------------- .../org/apache/comet/CometExecIterator.scala | 32 ++------- .../main/scala/org/apache/comet/Native.scala | 10 +-- 3 files changed, 28 insertions(+), 83 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 083744f0a4..8afe134cd3 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -31,8 +31,8 @@ use futures::poll; use jni::{ errors::Result as JNIResult, objects::{ - JByteArray, JClass, JIntArray, JLongArray, JMap, JObject, JObjectArray, JPrimitiveArray, - JString, ReleaseMode, + JByteArray, JClass, JIntArray, JLongArray, JObject, JObjectArray, JPrimitiveArray, JString, + ReleaseMode, }, sys::{jbyteArray, jint, jlong, jlongArray}, JNIEnv, @@ -77,8 +77,6 @@ struct ExecutionContext { pub input_sources: Vec>, /// The record batch stream to pull results from pub stream: Option, - /// Configurations for DF execution - pub conf: HashMap, /// The Tokio runtime used for async. pub runtime: Runtime, /// Native metrics @@ -103,11 +101,15 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( e: JNIEnv, _class: JClass, id: jlong, - config_object: JObject, iterators: jobjectArray, serialized_query: jbyteArray, metrics_node: JObject, comet_task_memory_manager_obj: JObject, + batch_size: jint, + debug_native: jboolean, + explain_native: jboolean, + worker_threads: jint, + blocking_threads: jint, ) -> jlong { try_unwrap_or_throw(&e, |mut env| { // Init JVM classes @@ -121,36 +123,10 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( // Deserialize query plan let spark_plan = serde::deserialize_op(bytes.as_slice())?; - // Sets up context - let mut configs = HashMap::new(); - - let config_map = JMap::from_env(&mut env, &config_object)?; - let mut map_iter = config_map.iter(&mut env)?; - while let Some((key, value)) = map_iter.next(&mut env)? { - let key: String = env.get_string(&JString::from(key)).unwrap().into(); - let value: String = env.get_string(&JString::from(value)).unwrap().into(); - configs.insert(key, value); - } - - // Whether we've enabled additional debugging on the native side - let debug_native = parse_bool(&configs, "debug_native")?; - let explain_native = parse_bool(&configs, "explain_native")?; - - let worker_threads = configs - .get("worker_threads") - .map(String::as_str) - .unwrap_or("4") - .parse::()?; - let blocking_threads = configs - .get("blocking_threads") - .map(String::as_str) - .unwrap_or("10") - .parse::()?; - // Use multi-threaded tokio runtime to prevent blocking spawned tasks if any let runtime = tokio::runtime::Builder::new_multi_thread() - .worker_threads(worker_threads) - .max_blocking_threads(blocking_threads) + .worker_threads(worker_threads as usize) + .max_blocking_threads(blocking_threads as usize) .enable_all() .build()?; @@ -171,7 +147,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( // We need to keep the session context alive. Some session state like temporary // dictionaries are stored in session context. If it is dropped, the temporary // dictionaries will be dropped as well. - let session = prepare_datafusion_session_context(&configs, task_memory_manager)?; + let session = prepare_datafusion_session_context(batch_size as usize, task_memory_manager)?; let plan_creation_time = start.elapsed(); @@ -182,13 +158,12 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( scans: vec![], input_sources, stream: None, - conf: configs, runtime, metrics, plan_creation_time, session_ctx: Arc::new(session), - debug_native, - explain_native, + debug_native: debug_native == 1, + explain_native: explain_native == 1, metrics_jstrings: HashMap::new(), }); @@ -196,19 +171,11 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( }) } -/// Parse Comet configs and configure DataFusion session context. +/// Configure DataFusion session context. fn prepare_datafusion_session_context( - conf: &HashMap, + batch_size: usize, comet_task_memory_manager: Arc, ) -> CometResult { - // Get the batch size from Comet JVM side - let batch_size = conf - .get("batch_size") - .ok_or(CometError::Internal( - "Config 'batch_size' is not specified from Comet JVM side".to_string(), - ))? - .parse::()?; - let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); // Set Comet memory pool for native @@ -218,7 +185,7 @@ fn prepare_datafusion_session_context( // Get Datafusion configuration from Spark Execution context // can be configured in Comet Spark JVM using Spark --conf parameters // e.g: spark-shell --conf spark.datafusion.sql_parser.parse_float_as_decimal=true - let mut session_config = SessionConfig::new() + let session_config = SessionConfig::new() .with_batch_size(batch_size) // DataFusion partial aggregates can emit duplicate rows so we disable the // skip partial aggregation feature because this is not compatible with Spark's @@ -231,11 +198,7 @@ fn prepare_datafusion_session_context( &ScalarValue::Float64(Some(1.1)), ); - for (key, value) in conf.iter().filter(|(k, _)| k.starts_with("datafusion.")) { - session_config = session_config.set_str(key, value); - } - - let runtime = RuntimeEnv::try_new(rt_config).unwrap(); + let runtime = RuntimeEnv::try_new(rt_config)?; let mut session_ctx = SessionContext::new_with_config_rt(session_config, Arc::new(runtime)); diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index bff3e79256..d57e9e2b87 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -60,43 +60,23 @@ class CometExecIterator( new CometBatchIterator(iterator, nativeUtil) }.toArray private val plan = { - val configs = createNativeConf nativeLib.createPlan( id, - configs, cometBatchIterators, protobufQueryPlan, nativeMetrics, - new CometTaskMemoryManager(id)) + new CometTaskMemoryManager(id), + batchSize = COMET_BATCH_SIZE.get(), + debug = COMET_DEBUG_ENABLED.get(), + explain = COMET_EXPLAIN_NATIVE_ENABLED.get(), + workerThreads = COMET_WORKER_THREADS.get(), + blockingThreads = COMET_BLOCKING_THREADS.get()) } private var nextBatch: Option[ColumnarBatch] = None private var currentBatch: ColumnarBatch = null private var closed: Boolean = false - /** - * Creates a new configuration map to be passed to the native side. - */ - private def createNativeConf: java.util.HashMap[String, String] = { - val result = new java.util.HashMap[String, String]() - val conf = SparkEnv.get.conf - - result.put("batch_size", String.valueOf(COMET_BATCH_SIZE.get())) - result.put("debug_native", String.valueOf(COMET_DEBUG_ENABLED.get())) - result.put("explain_native", String.valueOf(COMET_EXPLAIN_NATIVE_ENABLED.get())) - result.put("worker_threads", String.valueOf(COMET_WORKER_THREADS.get())) - result.put("blocking_threads", String.valueOf(COMET_BLOCKING_THREADS.get())) - - // Strip mandatory prefix spark. which is not required for DataFusion session params - conf.getAll.foreach { - case (k, v) if k.startsWith("spark.datafusion") => - result.put(k.replaceFirst("spark\\.", ""), v) - case _ => - } - - result - } - def getNextBatch(): Option[ColumnarBatch] = { assert(partitionIndex >= 0 && partitionIndex < numParts) diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 52063419df..64ada91adc 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -19,8 +19,6 @@ package org.apache.comet -import java.util.Map - import org.apache.spark.CometTaskMemoryManager import org.apache.spark.sql.comet.CometMetricNode @@ -47,11 +45,15 @@ class Native extends NativeBase { */ @native def createPlan( id: Long, - configMap: Map[String, String], iterators: Array[CometBatchIterator], plan: Array[Byte], metrics: CometMetricNode, - taskMemoryManager: CometTaskMemoryManager): Long + taskMemoryManager: CometTaskMemoryManager, + batchSize: Int, + debug: Boolean, + explain: Boolean, + workerThreads: Int, + blockingThreads: Int): Long /** * Execute a native query plan based on given input Arrow arrays. From 8d7bcb84beab70f46321eb4feeb21ebdd6b55709 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Dec 2024 17:01:50 -0700 Subject: [PATCH 23/47] feat: Improve ScanExec native metrics (#1133) * save * remove shuffle jvm metric and update tuning guide * docs * add source for all ScanExecs * address feedback * address feedback --- docs/source/user-guide/tuning.md | 21 ++++++--- .../execution/datafusion/shuffle_writer.rs | 26 +++-------- native/core/src/execution/operators/scan.rs | 44 ++++++++++++++++--- native/core/src/jvm_bridge/batch_iterator.rs | 4 ++ .../org/apache/comet/CometBatchIterator.java | 28 +++++++++--- .../apache/comet/serde/QueryPlanSerde.scala | 7 ++- .../spark/sql/comet/CometExecUtils.scala | 4 +- .../shuffle/CometShuffleExchangeExec.scala | 14 +----- 8 files changed, 97 insertions(+), 51 deletions(-) diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index e0d7c1bc93..af722494f9 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -103,7 +103,9 @@ native shuffle currently only supports `HashPartitioning` and `SinglePartitionin To enable native shuffle, set `spark.comet.exec.shuffle.mode` to `native`. If this mode is explicitly set, then any shuffle operations that cannot be supported in this mode will fall back to Spark. -## Metrics +## Metrics + +### Spark SQL Metrics Some Comet metrics are not directly comparable to Spark metrics in some cases: @@ -111,10 +113,17 @@ Some Comet metrics are not directly comparable to Spark metrics in some cases: milliseconds _per batch_ which can result in a large loss of precision, making it difficult to compare scan times between Spark and Comet. -Comet also adds some custom metrics: +### Native Metrics + +Setting `spark.comet.explain.native.enabled=true` will cause native plans to be logged in each executor. Metrics are +logged for each native plan (and there is one plan per task, so this is very verbose). + +Here is a guide to some of the native metrics. -### ShuffleWriterExec +### ScanExec -| Metric | Description | -| ---------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| `jvm_fetch_time` | Measure the time it takes for `ShuffleWriterExec` to fetch batches from the JVM. Note that this does not include the execution time of the query that produced the input batches. | +| Metric | Description | +| ----------------- | --------------------------------------------------------------------------------------------------- | +| `elapsed_compute` | Total time spent in this operator, fetching batches from a JVM iterator. | +| `jvm_fetch_time` | Time spent in the JVM fetching input batches to be read by this `ScanExec` instance. | +| `arrow_ffi_time` | Time spent using Arrow FFI to create Arrow batches from the memory addresses returned from the JVM. | diff --git a/native/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/datafusion/shuffle_writer.rs index c79eeeb4a0..7587ff06dc 100644 --- a/native/core/src/execution/datafusion/shuffle_writer.rs +++ b/native/core/src/execution/datafusion/shuffle_writer.rs @@ -139,7 +139,6 @@ impl ExecutionPlan for ShuffleWriterExec { ) -> Result { let input = self.input.execute(partition, Arc::clone(&context))?; let metrics = ShuffleRepartitionerMetrics::new(&self.metrics, 0); - let jvm_fetch_time = MetricBuilder::new(&self.metrics).subset_time("jvm_fetch_time", 0); Ok(Box::pin(RecordBatchStreamAdapter::new( self.schema(), @@ -152,7 +151,6 @@ impl ExecutionPlan for ShuffleWriterExec { self.partitioning.clone(), metrics, context, - jvm_fetch_time, ) .map_err(|e| ArrowError::ExternalError(Box::new(e))), ) @@ -1085,7 +1083,6 @@ impl Debug for ShuffleRepartitioner { } } -#[allow(clippy::too_many_arguments)] async fn external_shuffle( mut input: SendableRecordBatchStream, partition_id: usize, @@ -1094,7 +1091,6 @@ async fn external_shuffle( partitioning: Partitioning, metrics: ShuffleRepartitionerMetrics, context: Arc, - jvm_fetch_time: Time, ) -> Result { let schema = input.schema(); let mut repartitioner = ShuffleRepartitioner::new( @@ -1108,23 +1104,13 @@ async fn external_shuffle( context.session_config().batch_size(), ); - loop { - let mut timer = jvm_fetch_time.timer(); - let b = input.next().await; - timer.stop(); - - match b { - Some(batch_result) => { - // Block on the repartitioner to insert the batch and shuffle the rows - // into the corresponding partition buffer. - // Otherwise, pull the next batch from the input stream might overwrite the - // current batch in the repartitioner. - block_on(repartitioner.insert_batch(batch_result?))?; - } - _ => break, - } + while let Some(batch) = input.next().await { + // Block on the repartitioner to insert the batch and shuffle the rows + // into the corresponding partition buffer. + // Otherwise, pull the next batch from the input stream might overwrite the + // current batch in the repartitioner. + block_on(repartitioner.insert_batch(batch?))?; } - repartitioner.shuffle_write().await } diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 2cb8a84d94..a97caf0db1 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -77,6 +77,10 @@ pub struct ScanExec { metrics: ExecutionPlanMetricsSet, /// Baseline metrics baseline_metrics: BaselineMetrics, + /// Time waiting for JVM input plan to execute and return batches + jvm_fetch_time: Time, + /// Time spent in FFI + arrow_ffi_time: Time, } impl ScanExec { @@ -88,6 +92,8 @@ impl ScanExec { ) -> Result { let metrics_set = ExecutionPlanMetricsSet::default(); let baseline_metrics = BaselineMetrics::new(&metrics_set, 0); + let arrow_ffi_time = MetricBuilder::new(&metrics_set).subset_time("arrow_ffi_time", 0); + let jvm_fetch_time = MetricBuilder::new(&metrics_set).subset_time("jvm_fetch_time", 0); // Scan's schema is determined by the input batch, so we need to set it before execution. // Note that we determine if arrays are dictionary-encoded based on the @@ -97,8 +103,13 @@ impl ScanExec { // Dictionary-encoded primitive arrays are always unpacked. let first_batch = if let Some(input_source) = input_source.as_ref() { let mut timer = baseline_metrics.elapsed_compute().timer(); - let batch = - ScanExec::get_next(exec_context_id, input_source.as_obj(), data_types.len())?; + let batch = ScanExec::get_next( + exec_context_id, + input_source.as_obj(), + data_types.len(), + &jvm_fetch_time, + &arrow_ffi_time, + )?; timer.stop(); batch } else { @@ -124,6 +135,8 @@ impl ScanExec { cache, metrics: metrics_set, baseline_metrics, + jvm_fetch_time, + arrow_ffi_time, schema, }) } @@ -171,6 +184,8 @@ impl ScanExec { self.exec_context_id, self.input_source.as_ref().unwrap().as_obj(), self.data_types.len(), + &self.jvm_fetch_time, + &self.arrow_ffi_time, )?; *current_batch = Some(next_batch); } @@ -185,6 +200,8 @@ impl ScanExec { exec_context_id: i64, iter: &JObject, num_cols: usize, + jvm_fetch_time: &Time, + arrow_ffi_time: &Time, ) -> Result { if exec_context_id == TEST_EXEC_CONTEXT_ID { // This is a unit test. We don't need to call JNI. @@ -200,6 +217,21 @@ impl ScanExec { let mut env = JVMClasses::get_env()?; + let mut timer = jvm_fetch_time.timer(); + + let num_rows: i32 = unsafe { + jni_call!(&mut env, + comet_batch_iterator(iter).has_next() -> i32)? + }; + + timer.stop(); + + if num_rows == -1 { + return Ok(InputBatch::EOF); + } + + let mut timer = arrow_ffi_time.timer(); + let mut array_addrs = Vec::with_capacity(num_cols); let mut schema_addrs = Vec::with_capacity(num_cols); @@ -233,9 +265,9 @@ impl ScanExec { comet_batch_iterator(iter).next(array_obj, schema_obj) -> i32)? }; - if num_rows == -1 { - return Ok(InputBatch::EOF); - } + // we already checked for end of results on call to has_next() so should always + // have a valid row count when calling next() + assert!(num_rows != -1); let mut inputs: Vec = Vec::with_capacity(num_cols); @@ -255,6 +287,8 @@ impl ScanExec { } } + timer.stop(); + Ok(InputBatch::new(inputs, Some(num_rows as usize))) } } diff --git a/native/core/src/jvm_bridge/batch_iterator.rs b/native/core/src/jvm_bridge/batch_iterator.rs index 4870624d2b..45b10cf208 100644 --- a/native/core/src/jvm_bridge/batch_iterator.rs +++ b/native/core/src/jvm_bridge/batch_iterator.rs @@ -26,6 +26,8 @@ use jni::{ /// A struct that holds all the JNI methods and fields for JVM `CometBatchIterator` class. pub struct CometBatchIterator<'a> { pub class: JClass<'a>, + pub method_has_next: JMethodID, + pub method_has_next_ret: ReturnType, pub method_next: JMethodID, pub method_next_ret: ReturnType, } @@ -38,6 +40,8 @@ impl<'a> CometBatchIterator<'a> { Ok(CometBatchIterator { class, + method_has_next: env.get_method_id(Self::JVM_CLASS, "hasNext", "()I")?, + method_has_next_ret: ReturnType::Primitive(Primitive::Int), method_next: env.get_method_id(Self::JVM_CLASS, "next", "([J[J)I")?, method_next_ret: ReturnType::Primitive(Primitive::Int), }) diff --git a/spark/src/main/java/org/apache/comet/CometBatchIterator.java b/spark/src/main/java/org/apache/comet/CometBatchIterator.java index accd57c208..e05bea1dff 100644 --- a/spark/src/main/java/org/apache/comet/CometBatchIterator.java +++ b/spark/src/main/java/org/apache/comet/CometBatchIterator.java @@ -33,12 +33,31 @@ public class CometBatchIterator { final Iterator input; final NativeUtil nativeUtil; + private ColumnarBatch currentBatch = null; CometBatchIterator(Iterator input, NativeUtil nativeUtil) { this.input = input; this.nativeUtil = nativeUtil; } + /** + * Fetch the next input batch. + * + * @return Number of rows in next batch or -1 if no batches left. + */ + public int hasNext() { + if (currentBatch == null) { + if (input.hasNext()) { + currentBatch = input.next(); + } + } + if (currentBatch == null) { + return -1; + } else { + return currentBatch.numRows(); + } + } + /** * Get the next batches of Arrow arrays. * @@ -47,12 +66,11 @@ public class CometBatchIterator { * @return the number of rows of the current batch. -1 if there is no more batch. */ public int next(long[] arrayAddrs, long[] schemaAddrs) { - boolean hasBatch = input.hasNext(); - - if (!hasBatch) { + if (currentBatch == null) { return -1; } - - return nativeUtil.exportBatch(arrayAddrs, schemaAddrs, input.next()); + int numRows = nativeUtil.exportBatch(arrayAddrs, schemaAddrs, currentBatch); + currentBatch = null; + return numRows; } } diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 2bb467af58..b33f6b5a6b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2909,7 +2909,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case op if isCometSink(op) && op.output.forall(a => supportedDataType(a.dataType, true)) => // These operators are source of Comet native execution chain val scanBuilder = OperatorOuterClass.Scan.newBuilder() - scanBuilder.setSource(op.simpleStringWithNodeId()) + val source = op.simpleStringWithNodeId() + if (source.isEmpty) { + scanBuilder.setSource(op.getClass.getSimpleName) + } else { + scanBuilder.setSource(source) + } val scanTypes = op.output.flatten { attr => serializeDataType(attr.dataType) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecUtils.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecUtils.scala index 9698dc98b8..2fc73bb7c5 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecUtils.scala @@ -88,7 +88,7 @@ object CometExecUtils { * child partition */ def getLimitNativePlan(outputAttributes: Seq[Attribute], limit: Int): Option[Operator] = { - val scanBuilder = OperatorOuterClass.Scan.newBuilder() + val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource("LimitInput") val scanOpBuilder = OperatorOuterClass.Operator.newBuilder() val scanTypes = outputAttributes.flatten { attr => @@ -118,7 +118,7 @@ object CometExecUtils { sortOrder: Seq[SortOrder], child: SparkPlan, limit: Int): Option[Operator] = { - val scanBuilder = OperatorOuterClass.Scan.newBuilder() + val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource("TopKInput") val scanOpBuilder = OperatorOuterClass.Operator.newBuilder() val scanTypes = outputAttributes.flatten { attr => diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index a7a33c40d6..b1dd9ac836 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -77,9 +77,6 @@ case class CometShuffleExchangeExec( SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics: Map[String, SQLMetric] = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), - "jvm_fetch_time" -> SQLMetrics.createNanoTimingMetric( - sparkContext, - "time fetching batches from JVM"), "numPartitions" -> SQLMetrics.createMetric( sparkContext, "number of partitions")) ++ readMetrics ++ writeMetrics @@ -485,14 +482,7 @@ class CometShuffleWriteProcessor( "output_rows" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN), "data_size" -> metrics("dataSize"), "elapsed_compute" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME)) - - val nativeMetrics = if (metrics.contains("jvm_fetch_time")) { - CometMetricNode( - nativeSQLMetrics ++ Map("jvm_fetch_time" -> - metrics("jvm_fetch_time"))) - } else { - CometMetricNode(nativeSQLMetrics) - } + val nativeMetrics = CometMetricNode(nativeSQLMetrics) // Getting rid of the fake partitionId val newInputs = inputs.asInstanceOf[Iterator[_ <: Product2[Any, Any]]].map(_._2) @@ -538,7 +528,7 @@ class CometShuffleWriteProcessor( } def getNativePlan(dataFile: String, indexFile: String): Operator = { - val scanBuilder = OperatorOuterClass.Scan.newBuilder() + val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource("ShuffleWriterInput") val opBuilder = OperatorOuterClass.Operator.newBuilder() val scanTypes = outputAttributes.flatten { attr => From 587c29bfb10e699070d6ea2ca27cda767bc31e51 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Dec 2024 18:02:18 -0700 Subject: [PATCH 24/47] chore: Remove unused StringView struct (#1143) * Remove unused StringView struct * remove more dead code --- native/core/src/common/mod.rs | 18 --- native/core/src/data_type.rs | 241 ---------------------------------- native/core/src/lib.rs | 3 - 3 files changed, 262 deletions(-) delete mode 100644 native/core/src/data_type.rs diff --git a/native/core/src/common/mod.rs b/native/core/src/common/mod.rs index 1b7dfad287..dc539879fa 100644 --- a/native/core/src/common/mod.rs +++ b/native/core/src/common/mod.rs @@ -17,23 +17,5 @@ #[macro_use] pub mod bit; - -use crate::TypeTrait; - -/// Getter APIs for Comet vectors. -trait ValueGetter { - /// Gets the non-null value at `idx`. - /// - /// Note that null check needs to be done before the call, to ensure the value at `idx` is - /// not null. - fn value(&self, idx: usize) -> T::Native; -} - -/// Setter APIs for Comet mutable vectors. -trait ValueSetter { - /// Appends a non-null value `v` to the end of this vector. - fn append_value(&mut self, v: &T::Native); -} - mod buffer; pub use buffer::*; diff --git a/native/core/src/data_type.rs b/native/core/src/data_type.rs deleted file mode 100644 index b275de1c65..0000000000 --- a/native/core/src/data_type.rs +++ /dev/null @@ -1,241 +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. - -use arrow::datatypes::DataType as ArrowDataType; -use arrow_schema::TimeUnit; -use std::{cmp, fmt::Debug}; - -#[derive(Debug, PartialEq)] -pub enum DataType { - Boolean, - Byte, - Short, - Integer, - Long, - Float, - Double, - Decimal(u8, i8), - String, - Binary, - Timestamp, - Date, -} - -impl From<&ArrowDataType> for DataType { - fn from(dt: &ArrowDataType) -> Self { - match dt { - ArrowDataType::Boolean => DataType::Boolean, - ArrowDataType::Int8 => DataType::Byte, - ArrowDataType::Int16 => DataType::Short, - ArrowDataType::Int32 => DataType::Integer, - ArrowDataType::Int64 => DataType::Long, - ArrowDataType::Float32 => DataType::Float, - ArrowDataType::Float64 => DataType::Double, - ArrowDataType::Decimal128(precision, scale) => DataType::Decimal(*precision, *scale), - ArrowDataType::Utf8 => DataType::String, - ArrowDataType::Binary => DataType::Binary, - // Spark always store timestamp in micro seconds - ArrowDataType::Timestamp(TimeUnit::Microsecond, _) => DataType::Timestamp, - ArrowDataType::Date32 => DataType::Date, - ArrowDataType::Dictionary(key_dt, value_dt) if is_valid_key_type(key_dt) => { - Self::from(value_dt.as_ref()) - } - dt => panic!("unsupported Arrow data type: {:?}", dt), - } - } -} - -impl DataType { - pub fn kind(&self) -> TypeKind { - match self { - DataType::Boolean => TypeKind::Boolean, - DataType::Byte => TypeKind::Byte, - DataType::Short => TypeKind::Short, - DataType::Integer => TypeKind::Integer, - DataType::Long => TypeKind::Long, - DataType::Float => TypeKind::Float, - DataType::Double => TypeKind::Double, - DataType::Decimal(_, _) => TypeKind::Decimal, - DataType::String => TypeKind::String, - DataType::Binary => TypeKind::Binary, - DataType::Timestamp => TypeKind::Timestamp, - DataType::Date => TypeKind::Date, - } - } -} - -/// Comet only use i32 as dictionary key -fn is_valid_key_type(dt: &ArrowDataType) -> bool { - matches!(dt, ArrowDataType::Int32) -} - -/// Unlike [`DataType`], [`TypeKind`] doesn't carry extra information about the type itself, such as -/// decimal precision & scale. Instead, it is merely a token that is used to do runtime case -/// analysis depending on the actual type. It can be obtained from a `TypeTrait` generic parameter. -#[derive(Debug, PartialEq)] -pub enum TypeKind { - Boolean, - Byte, - Short, - Integer, - Long, - Float, - Double, - Decimal, - String, - Binary, - Timestamp, - Date, -} - -pub const BITS_PER_BYTE: usize = 8; - -impl TypeKind { - /// Returns the size of this type, in number of bits. - pub fn type_size(&self) -> usize { - match self { - TypeKind::Boolean => 1, - TypeKind::Byte => BITS_PER_BYTE, - TypeKind::Short => BITS_PER_BYTE * 2, - TypeKind::Integer | TypeKind::Float => BITS_PER_BYTE * 4, - TypeKind::Long | TypeKind::Double => BITS_PER_BYTE * 8, - TypeKind::Decimal => BITS_PER_BYTE * 16, - TypeKind::String | TypeKind::Binary => BITS_PER_BYTE * 16, - TypeKind::Timestamp => BITS_PER_BYTE * 8, - TypeKind::Date => BITS_PER_BYTE * 4, - } - } -} - -pub const STRING_VIEW_LEN: usize = 16; // StringView is stored using 16 bytes -pub const STRING_VIEW_PREFIX_LEN: usize = 4; // String prefix in StringView is stored using 4 bytes - -#[repr(C, align(16))] -#[derive(Clone, Copy, Debug)] -pub struct StringView { - pub len: u32, - pub prefix: [u8; STRING_VIEW_PREFIX_LEN], - pub ptr: usize, -} - -impl StringView { - pub fn as_utf8_str(&self) -> &str { - unsafe { - let slice = std::slice::from_raw_parts(self.ptr as *const u8, self.len as usize); - std::str::from_utf8_unchecked(slice) - } - } -} - -impl Default for StringView { - fn default() -> Self { - Self { - len: 0, - prefix: [0; STRING_VIEW_PREFIX_LEN], - ptr: 0, - } - } -} - -impl PartialEq for StringView { - fn eq(&self, other: &Self) -> bool { - if self.len != other.len { - return false; - } - if self.prefix != other.prefix { - return false; - } - self.as_utf8_str() == other.as_utf8_str() - } -} - -pub trait NativeEqual { - fn is_equal(&self, other: &Self) -> bool; -} - -macro_rules! make_native_equal { - ($native_ty:ty) => { - impl NativeEqual for $native_ty { - fn is_equal(&self, other: &Self) -> bool { - self == other - } - } - }; -} - -make_native_equal!(bool); -make_native_equal!(i8); -make_native_equal!(i16); -make_native_equal!(i32); -make_native_equal!(i64); -make_native_equal!(i128); -make_native_equal!(StringView); - -impl NativeEqual for f32 { - fn is_equal(&self, other: &Self) -> bool { - self.total_cmp(other) == cmp::Ordering::Equal - } -} - -impl NativeEqual for f64 { - fn is_equal(&self, other: &Self) -> bool { - self.total_cmp(other) == cmp::Ordering::Equal - } -} -pub trait NativeType: Debug + Default + Copy + NativeEqual {} - -impl NativeType for bool {} -impl NativeType for i8 {} -impl NativeType for i16 {} -impl NativeType for i32 {} -impl NativeType for i64 {} -impl NativeType for i128 {} -impl NativeType for f32 {} -impl NativeType for f64 {} -impl NativeType for StringView {} - -/// A trait for Comet data type. This should only be used as generic parameter during method -/// invocations. -pub trait TypeTrait: 'static { - type Native: NativeType; - fn type_kind() -> TypeKind; -} - -macro_rules! make_type_trait { - ($name:ident, $native_ty:ty, $kind:path) => { - pub struct $name {} - impl TypeTrait for $name { - type Native = $native_ty; - fn type_kind() -> TypeKind { - $kind - } - } - }; -} - -make_type_trait!(BoolType, bool, TypeKind::Boolean); -make_type_trait!(ByteType, i8, TypeKind::Byte); -make_type_trait!(ShortType, i16, TypeKind::Short); -make_type_trait!(IntegerType, i32, TypeKind::Integer); -make_type_trait!(LongType, i64, TypeKind::Long); -make_type_trait!(FloatType, f32, TypeKind::Float); -make_type_trait!(DoubleType, f64, TypeKind::Double); -make_type_trait!(DecimalType, i128, TypeKind::Decimal); -make_type_trait!(StringType, StringView, TypeKind::String); -make_type_trait!(BinaryType, StringView, TypeKind::Binary); -make_type_trait!(TimestampType, i64, TypeKind::Timestamp); -make_type_trait!(DateType, i32, TypeKind::Date); diff --git a/native/core/src/lib.rs b/native/core/src/lib.rs index 36e63e39c9..c6a7a4143d 100644 --- a/native/core/src/lib.rs +++ b/native/core/src/lib.rs @@ -40,15 +40,12 @@ use log4rs::{ use mimalloc::MiMalloc; use once_cell::sync::OnceCell; -pub use data_type::*; - use errors::{try_unwrap_or_throw, CometError, CometResult}; #[macro_use] mod errors; #[macro_use] pub mod common; -mod data_type; pub mod execution; mod jvm_bridge; pub mod parquet; From b95dc1d84554b7cd5728ed53a24463a2e111ba44 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Dec 2024 13:04:51 -0700 Subject: [PATCH 25/47] docs: Add some documentation explaining how shuffle works (#1148) * add some notes on shuffle * reads * improve docs --- .../contributor-guide/plugin_overview.md | 59 +++++++++++++------ 1 file changed, 42 insertions(+), 17 deletions(-) diff --git a/docs/source/contributor-guide/plugin_overview.md b/docs/source/contributor-guide/plugin_overview.md index 3e7c24f55a..add4172d5c 100644 --- a/docs/source/contributor-guide/plugin_overview.md +++ b/docs/source/contributor-guide/plugin_overview.md @@ -19,17 +19,26 @@ under the License. # Comet Plugin Architecture +## Overview + +The Comet plugin enhances Spark SQL by introducing optimized query execution and shuffle mechanisms leveraging +native code. It integrates with Spark's plugin framework and extension API to replace or extend Spark's +default behavior. + +--- + +# Plugin Components + ## Comet SQL Plugin -The entry point to Comet is the `org.apache.spark.CometPlugin` class, which can be registered with Spark by adding the -following setting to the Spark configuration when launching `spark-shell` or `spark-submit`: +The entry point to Comet is the org.apache.spark.CometPlugin class, which is registered in Spark using the following +configuration: ``` --conf spark.plugins=org.apache.spark.CometPlugin ``` -This class is loaded by Spark's plugin framework. It will be instantiated in the Spark driver only. Comet does not -provide any executor plugins. +The plugin is loaded on the Spark driver and does not provide executor-side plugins. The plugin will update the current `SparkConf` with the extra configuration provided by Comet, such as executor memory configuration. @@ -87,7 +96,7 @@ In the native code there is a `PhysicalPlanner` struct (in `planner.rs`) which c Apache DataFusion `ExecutionPlan`. In some cases, Comet provides specialized physical operators and expressions to override the DataFusion versions to ensure compatibility with Apache Spark. -The leaf nodes in the physical plan are always `ScanExec` and each of these operators will make a JNI call to +The leaf nodes in the physical plan are always `ScanExec` and each of these operators will make a JNI call to `CometBatchIterator.next()` to fetch the next input batch. The input could be a Comet native Parquet scan, a Spark exchange, or another native plan. @@ -95,15 +104,38 @@ a Spark exchange, or another native plan. partitions. Each call to `CometExecIterator.next()` will invoke `Native.executePlan`. Once the plan finishes executing, the resulting Arrow batches are imported into the JVM using Arrow FFI. -## Arrow +## Shuffle -Due to the hybrid execution model, it is necessary to pass batches of data between the JVM and native code. +Comet integrates with Spark's shuffle mechanism, optimizing both shuffle writes and reads. Comet's shuffle manager +must be registered with Spark using the following configuration: + +``` +--conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager +``` + +### Shuffle Writes -Comet uses a combination of Arrow FFI and Arrow IPC to achieve this. +For shuffle writes, a `ShuffleMapTask` runs in the executors. This task contains a `ShuffleDependency` that is +broadcast to all of the executors. It then passes the input RDD to `ShuffleWriteProcessor.write()` which +requests a `ShuffleWriter` from the shuffle manager, and this is where it gets a Comet shuffle writer. -### Arrow FFI +`ShuffleWriteProcessor` then invokes the dependency RDD and fetches rows/batches and passes them to Comet's +shuffle writer, which writes batches to disk in Arrow IPC format. -The foundation for Arrow FFI is the [Arrow C Data Interface], which provides a stable ABI-compatible interface for +As a result, we cannot avoid having one native plan to produce the shuffle input and another native plan for +writing the batches to the shuffle file. + +### Shuffle Reads + +For shuffle reads a `ShuffledRDD` requests a `ShuffleReader` from the shuffle manager. Comet provides a +`CometBlockStoreShuffleReader` which is implemented in JVM and fetches blocks from Spark and then creates an +`ArrowReaderIterator` to process the blocks using Arrow's `StreamReader` for decoding IPC batches. + +## Arrow FFI + +Due to the hybrid execution model, it is necessary to pass batches of data between the JVM and native code. + +The foundation for Arrow FFI is the [Arrow C Data Interface], which provides a stable ABI-compatible interface for accessing Arrow data structures from multiple languages. [Arrow C Data Interface]: https://arrow.apache.org/docs/format/CDataInterface.html @@ -111,13 +143,6 @@ accessing Arrow data structures from multiple languages. - `CometExecIterator` invokes native plans and uses Arrow FFI to read the output batches - Native `ScanExec` operators call `CometBatchIterator` via JNI to fetch input batches from the JVM -### Arrow IPC - -Comet native shuffle uses Arrow IPC to write batches to the shuffle files. - -- `CometShuffleWriteProcessor` invokes a native plan to fetch batches and then passes them to native `ShuffleWriterExec` -- `CometBlockStoreShuffleReader` reads batches from shuffle files - ## End to End Flow The following diagram shows an example of the end-to-end flow for a query stage. From 1c6c7a93ae1a04e82255768ef46e401ed57453d2 Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA Date: Fri, 6 Dec 2024 12:55:05 -0800 Subject: [PATCH 26/47] test: enable more Spark 4.0 tests (#1145) ## Which issue does this PR close? Part of https://github.com/apache/datafusion-comet/issues/372 and https://github.com/apache/datafusion-comet/issues/551 ## Rationale for this change To be ready for Spark 4.0 ## What changes are included in this PR? This PR enables more Spark 4.0 tests that were fixed by recent changes ## How are these changes tested? tests enabled --- dev/diffs/4.0.0-preview1.diff | 476 +--------------------------------- 1 file changed, 4 insertions(+), 472 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 335aa01d1e..1a6810a7ab 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index a4b1b2c3c9f..db50bdb0d3b 100644 +index a4b1b2c3c9f..6a532749978 100644 --- a/pom.xml +++ b/pom.xml @@ -147,6 +147,8 @@ @@ -38,7 +38,7 @@ index a4b1b2c3c9f..db50bdb0d3b 100644 org.apache.datasketches diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index 19f6303be36..31e1d27700f 100644 +index 19f6303be36..6c0e77882e6 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -146,133 +146,6 @@ index 698ca009b4f..57d774a3617 100644 -- Test tables CREATE table explain_temp1 (key int, val int) USING PARQUET; -diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql -index 1152d77da0c..f77493f690b 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql -@@ -7,6 +7,9 @@ - - -- avoid bit-exact output here because operations may not be bit-exact. - -- SET extra_float_digits = 0; -+-- Disable Comet exec due to floating point precision difference -+--SET spark.comet.exec.enabled = false -+ - - -- Test aggregate operator with codegen on and off. - --CONFIG_DIM1 spark.sql.codegen.wholeStage=true -diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql -index 41fd4de2a09..44cd244d3b0 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql -@@ -5,6 +5,9 @@ - -- AGGREGATES [Part 3] - -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605 - -+-- Disable Comet exec due to floating point precision difference -+--SET spark.comet.exec.enabled = false -+ - -- Test aggregate operator with codegen on and off. - --CONFIG_DIM1 spark.sql.codegen.wholeStage=true - --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY -diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql -index 932cdb95fcf..bbafaadb960 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql -@@ -5,6 +5,9 @@ - -- FLOAT8 - -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float8.sql - -+-- TODO: https://github.com/apache/datafusion-comet/issues/551 -+--SET spark.comet.enabled = false -+ - CREATE TABLE FLOAT8_TBL(f1 double) USING parquet; - - -- PostgreSQL implicitly casts string literals to data with floating point types, but -diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql -index f06b0276b00..1f39a1b3da8 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql -@@ -5,6 +5,9 @@ - - -- test data sources - -+-- TODO: https://github.com/apache/datafusion-comet/issues/551 -+--SET spark.comet.enabled = false -+ - create temp view gstest1(a,b,v) - as values (1,1,10),(1,1,11),(1,2,12),(1,2,13),(1,3,14), - (2,3,15), -diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql -index 3a409eea348..26e9aaf215c 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql -@@ -6,6 +6,9 @@ - -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int4.sql - -- - -+-- TODO: https://github.com/apache/datafusion-comet/issues/551 -+--SET spark.comet.enabled = false -+ - CREATE TABLE INT4_TBL(f1 int) USING parquet; - - -- [SPARK-28023] Trim the string when cast string type to other types -diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql -index fac23b4a26f..2b73732c33f 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql -@@ -1,6 +1,10 @@ - -- - -- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group - -- -+ -+-- Disable Comet exec due to floating point precision difference -+--SET spark.comet.exec.enabled = false -+ - -- - -- INT8 - -- Test int8 64-bit integers. -diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql -index 0efe0877e9b..423d3b3d76d 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql -@@ -1,6 +1,10 @@ - -- - -- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group - -- -+ -+-- Disable Comet exec due to floating point precision difference -+--SET spark.comet.exec.enabled = false -+ - -- - -- SELECT_HAVING - -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql -diff --git a/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql b/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql -index e803254ea64..74db78aee38 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql -@@ -1,6 +1,9 @@ - -- This test suits check the spark.sql.viewSchemaBindingMode configuration. - -- It can be DISABLED and COMPENSATION - -+-- TODO: https://github.com/apache/datafusion-comet/issues/551 -+--SET spark.comet.enabled = false -+ - -- Verify the default binding is true - SET spark.sql.legacy.viewSchemaBindingMode; - -diff --git a/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql b/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql -index 21a3ce1e122..316788b2989 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql -@@ -1,4 +1,8 @@ - -- This test suite checks the WITH SCHEMA COMPENSATION clause -+ -+-- TODO: https://github.com/apache/datafusion-comet/issues/551 -+--SET spark.comet.enabled = false -+ - -- Disable ANSI mode to ensure we are forcing it explicitly in the CASTS - SET spark.sql.ansi.enabled = false; - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index d023fb82185..0f4f03bda6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -348,7 +221,7 @@ index f6fd6b501d7..11870c85d82 100644 spark.range(100).write.saveAsTable(s"$dbName.$table2Name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -index 760ee802608..ebd4a34b08d 100644 +index 760ee802608..b77133ffd37 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -36,11 +36,12 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, @@ -407,16 +280,6 @@ index 760ee802608..ebd4a34b08d 100644 } assert(exchanges.size == 2) } -@@ -2299,7 +2302,8 @@ class DataFrameSuite extends QueryTest - assert(df2.isLocal) - } - -- test("SPARK-35886: PromotePrecision should be subexpr replaced") { -+ test("SPARK-35886: PromotePrecision should be subexpr replaced", -+ IgnoreComet("TODO: fix Comet for this test")) { - withTable("tbl") { - sql( - """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 16a493b5290..3f0b70e2d59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1054,7 +917,7 @@ index 34c6c49bc49..f5dea07a213 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 56c364e2084..fc3abd7cdc4 100644 +index 56c364e2084..a00a50e020a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1510,7 +1510,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1067,36 +930,6 @@ index 56c364e2084..fc3abd7cdc4 100644 AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } -@@ -4454,7 +4455,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark - } - - test("SPARK-39166: Query context of binary arithmetic should be serialized to executors" + -- " when WSCG is off") { -+ " when WSCG is off", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", - SQLConf.ANSI_ENABLED.key -> "true") { - withTable("t") { -@@ -4475,7 +4477,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark - } - - test("SPARK-39175: Query context of Cast should be serialized to executors" + -- " when WSCG is off") { -+ " when WSCG is off", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", - SQLConf.ANSI_ENABLED.key -> "true") { - withTable("t") { -@@ -4502,7 +4505,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark - } - - test("SPARK-39190,SPARK-39208,SPARK-39210: Query context of decimal overflow error should " + -- "be serialized to executors when WSCG is off") { -+ "be serialized to executors when WSCG is off", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", - SQLConf.ANSI_ENABLED.key -> "true") { - withTable("t") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 68f14f13bbd..174636cefb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2145,29 +1978,6 @@ index a7efd0aa75e..baae0967a2a 100644 }.isEmpty) assert(collect(initialExecutedPlan) { case i: InMemoryTableScanLike => i -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala -index 05872d41131..0dd83608bbd 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCustomMetadataStructSuite.scala -@@ -21,7 +21,7 @@ import java.io.File - - import org.apache.hadoop.fs.{FileStatus, Path} - --import org.apache.spark.sql.{DataFrame, Dataset, QueryTest, Row} -+import org.apache.spark.sql.{DataFrame, Dataset, IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.InternalRow - import org.apache.spark.sql.catalyst.expressions.{Expression, FileSourceConstantMetadataStructField, FileSourceGeneratedMetadataStructField, Literal} - import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -@@ -134,7 +134,8 @@ class FileSourceCustomMetadataStructSuite extends QueryTest with SharedSparkSess - } - } - -- test("[SPARK-43226] extra constant metadata fields with extractors") { -+ test("[SPARK-43226] extra constant metadata fields with extractors", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { - withTempData("parquet", FILE_SCHEMA) { (_, f0, f1) => - val format = new TestFileFormat(extraConstantMetadataFields) { - val extractPartitionNumber = { pf: PartitionedFile => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 0a0b23d1e60..5685926250f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -2541,35 +2351,6 @@ index 4bd35e0789b..6544d86dbe0 100644 ) } test(s"parquet widening conversion $fromType -> $toType") { -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala -index c800168b507..991d52a1a75 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala -@@ -22,7 +22,7 @@ import org.scalatest.Assertions - - import org.apache.spark.SparkUnsupportedOperationException - import org.apache.spark.io.CompressionCodec --import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row} -+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, IgnoreComet, Row} - import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow} - import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning - import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil -@@ -525,11 +525,13 @@ abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Ass - } - } - -- test("flatMapGroupsWithState, state ver 1") { -+ test("flatMapGroupsWithState, state ver 1", -+ IgnoreComet("Ignored if Comet is enabled due to SPARK-49070.")) { - testFlatMapGroupsWithState(1) - } - -- test("flatMapGroupsWithState, state ver 2") { -+ test("flatMapGroupsWithState, state ver 2", -+ IgnoreComet("Ignored if Comet is enabled due to SPARK-49070.")) { - testFlatMapGroupsWithState(2) - } - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index b8f3ea3c6f3..bbd44221288 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -2900,94 +2681,6 @@ index 04193d5189a..d83d03f8e0d 100644 }.headOption.getOrElse { fail(s"No FileScan in query\n${df.queryExecution}") } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala -index b597a244710..b2e8be41065 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala -@@ -21,6 +21,7 @@ import java.io.File - - import org.apache.commons.io.FileUtils - -+import org.apache.spark.sql.IgnoreComet - import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update - import org.apache.spark.sql.execution.streaming.{FlatMapGroupsWithStateExec, MemoryStream} - import org.apache.spark.sql.internal.SQLConf -@@ -91,7 +92,7 @@ class FlatMapGroupsWithStateDistributionSuite extends StreamTest - } - - test("SPARK-38204: flatMapGroupsWithState should require StatefulOpClusteredDistribution " + -- "from children - without initial state") { -+ "from children - without initial state", IgnoreComet("TODO: fix Comet for this test")) { - // function will return -1 on timeout and returns count of the state otherwise - val stateFunc = - (key: (String, String), values: Iterator[(String, String, Long)], -@@ -243,7 +244,8 @@ class FlatMapGroupsWithStateDistributionSuite extends StreamTest - } - - test("SPARK-38204: flatMapGroupsWithState should require ClusteredDistribution " + -- "from children if the query starts from checkpoint in 3.2.x - without initial state") { -+ "from children if the query starts from checkpoint in 3.2.x - without initial state", -+ IgnoreComet("TODO: fix Comet for this test")) { - // function will return -1 on timeout and returns count of the state otherwise - val stateFunc = - (key: (String, String), values: Iterator[(String, String, Long)], -@@ -335,7 +337,8 @@ class FlatMapGroupsWithStateDistributionSuite extends StreamTest - } - - test("SPARK-38204: flatMapGroupsWithState should require ClusteredDistribution " + -- "from children if the query starts from checkpoint in prior to 3.2") { -+ "from children if the query starts from checkpoint in prior to 3.2", -+ IgnoreComet("TODO: fix Comet for this test")) { - // function will return -1 on timeout and returns count of the state otherwise - val stateFunc = - (key: (String, String), values: Iterator[(String, String, Long)], -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala -index b35e996106f..326f72235e0 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala -@@ -25,7 +25,7 @@ import org.scalatest.exceptions.TestFailedException - - import org.apache.spark.SparkException - import org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction --import org.apache.spark.sql.{DataFrame, Encoder} -+import org.apache.spark.sql.{DataFrame, Encoder, IgnoreCometSuite} - import org.apache.spark.sql.catalyst.InternalRow - import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} - import org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsWithState -@@ -46,8 +46,9 @@ case class RunningCount(count: Long) - - case class Result(key: Long, count: Int) - -+// TODO: fix Comet to enable this suite - @SlowSQLTest --class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { -+class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with IgnoreCometSuite { - - import testImplicits._ - -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala -index 2a2a83d35e1..e3b7b290b3e 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala -@@ -18,7 +18,7 @@ - package org.apache.spark.sql.streaming - - import org.apache.spark.SparkException --import org.apache.spark.sql.{AnalysisException, Dataset, KeyValueGroupedDataset} -+import org.apache.spark.sql.{AnalysisException, Dataset, IgnoreComet, KeyValueGroupedDataset} - import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update - import org.apache.spark.sql.execution.streaming.MemoryStream - import org.apache.spark.sql.execution.streaming.state.FlatMapGroupsWithStateExecHelper -@@ -253,7 +253,8 @@ class FlatMapGroupsWithStateWithInitialStateSuite extends StateStoreMetricsTest - assert(e.message.contains(expectedError)) - } - -- test("flatMapGroupsWithState - initial state - initial state has flatMapGroupsWithState") { -+ test("flatMapGroupsWithState - initial state - initial state has flatMapGroupsWithState", -+ IgnoreComet("TODO: fix Comet for this test")) { - val initialStateDS = Seq(("keyInStateAndData", new RunningCount(1))).toDS() - val initialState: KeyValueGroupedDataset[String, RunningCount] = - initialStateDS.groupByKey(_._1).mapValues(_._2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 1fce992126b..6d3ea74e0fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -3102,153 +2795,6 @@ index e05cb4d3c35..dc65a4fe18e 100644 }) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala -index dea16e52989..55cdf47c4d5 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala -@@ -18,7 +18,7 @@ - package org.apache.spark.sql.streaming - - import org.apache.spark.SparkIllegalArgumentException --import org.apache.spark.sql.Encoders -+import org.apache.spark.sql.{Encoders, IgnoreCometSuite} - import org.apache.spark.sql.execution.streaming.MemoryStream - import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider} - import org.apache.spark.sql.internal.SQLConf -@@ -128,7 +128,7 @@ class ToggleSaveAndEmitProcessor - } - - class TransformWithListStateSuite extends StreamTest -- with AlsoTestWithChangelogCheckpointingEnabled { -+ with AlsoTestWithChangelogCheckpointingEnabled with IgnoreCometSuite { - import testImplicits._ - - test("test appending null value in list state throw exception") { -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala -index 299a3346b2e..2213f3c52a6 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala -@@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming - - import java.time.Duration - --import org.apache.spark.sql.Encoders -+import org.apache.spark.sql.{Encoders, IgnoreCometSuite} - import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, MemoryStream} - import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider - import org.apache.spark.sql.internal.SQLConf -@@ -95,7 +95,7 @@ class ListStateTTLProcessor(ttlConfig: TTLConfig) - * Test suite for testing list state with TTL. - * We use the base TTL suite with a list state processor. - */ --class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { -+class TransformWithListStateTTLSuite extends TransformWithStateTTLTest with IgnoreCometSuite { - - import testImplicits._ - -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala -index bf46c802fde..623c3003430 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala -@@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming - - import java.time.Duration - --import org.apache.spark.sql.Encoders -+import org.apache.spark.sql.{Encoders, IgnoreCometSuite} - import org.apache.spark.sql.execution.streaming.{MapStateImplWithTTL, MemoryStream} - import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider - import org.apache.spark.sql.internal.SQLConf -@@ -174,7 +174,7 @@ class MapStateTTLProcessor(ttlConfig: TTLConfig) - } - } - --class TransformWithMapStateTTLSuite extends TransformWithStateTTLTest { -+class TransformWithMapStateTTLSuite extends TransformWithStateTTLTest with IgnoreCometSuite { - - import testImplicits._ - override def getProcessor(ttlConfig: TTLConfig): -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala -index 5388d6f1fb6..8aa11c5b875 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala -@@ -21,7 +21,7 @@ import java.sql.Timestamp - import java.time.{Instant, LocalDateTime, ZoneId} - - import org.apache.spark.{SparkRuntimeException, SparkThrowable} --import org.apache.spark.sql.AnalysisException -+import org.apache.spark.sql.{AnalysisException, IgnoreCometSuite} - import org.apache.spark.sql.catalyst.ExtendedAnalysisException - import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} - import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider -@@ -106,7 +106,8 @@ case class AggEventRow( - window: Window, - count: Long) - --class TransformWithStateChainingSuite extends StreamTest { -+class TransformWithStateChainingSuite extends StreamTest -+ with IgnoreCometSuite { - import testImplicits._ - - test("watermark is propagated correctly for next stateful operator" + -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala -index 0057af44d3e..51975748309 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala -@@ -22,7 +22,7 @@ import java.util.UUID - - import org.apache.spark.SparkRuntimeException - import org.apache.spark.internal.Logging --import org.apache.spark.sql.{Dataset, Encoders} -+import org.apache.spark.sql.{Dataset, Encoders, IgnoreCometSuite} - import org.apache.spark.sql.catalyst.util.stringToFile - import org.apache.spark.sql.execution.streaming._ - import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider, StatefulProcessorCannotPerformOperationWithInvalidHandleState, StateStoreMultipleColumnFamiliesNotSupportedException} -@@ -307,9 +307,11 @@ class RunningCountStatefulProcessorWithError extends RunningCountStatefulProcess - - /** - * Class that adds tests for transformWithState stateful streaming operator -+ * -+ * Ignored if Comet is enabled due to SPARK-49070. - */ - class TransformWithStateSuite extends StateStoreMetricsTest -- with AlsoTestWithChangelogCheckpointingEnabled { -+ with AlsoTestWithChangelogCheckpointingEnabled with IgnoreCometSuite { - - import testImplicits._ - -@@ -786,7 +788,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest - } - } - --class TransformWithStateValidationSuite extends StateStoreMetricsTest { -+class TransformWithStateValidationSuite extends StateStoreMetricsTest with IgnoreCometSuite { - import testImplicits._ - - test("transformWithState - streaming with hdfsStateStoreProvider should fail") { -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala -index 54004b419f7..4e5b35aa0da 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala -@@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming - import java.time.Duration - - import org.apache.spark.internal.Logging --import org.apache.spark.sql.Encoders -+import org.apache.spark.sql.{Encoders, IgnoreCometSuite} - import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImpl, ValueStateImplWithTTL} - import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider - import org.apache.spark.sql.internal.SQLConf -@@ -160,7 +160,8 @@ case class MultipleValueStatesTTLProcessor( - } - } - --class TransformWithValueStateTTLSuite extends TransformWithStateTTLTest { -+class TransformWithValueStateTTLSuite extends TransformWithStateTTLTest -+ with IgnoreCometSuite{ - - import testImplicits._ - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index af07aceaed1..ed0b5e6d9be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -3423,20 +2969,6 @@ index 52abd248f3a..7a199931a08 100644 case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child } -diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala -index 4b000fff0eb..e867b6c0d9b 100644 ---- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala -+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala -@@ -660,7 +660,8 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te - Row(3, 4, 4, 3, null) :: Nil) - } - -- test("single distinct multiple columns set") { -+ test("single distinct multiple columns set", -+ IgnoreComet("TODO: fix Comet for this test")) { - checkAnswer( - spark.sql( - """ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 3f8de93b330..fc06e81f41a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala From 8d83cc15c87a14b93ebbec12d6fe4dce5283f4a2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Dec 2024 15:16:39 -0700 Subject: [PATCH 27/47] chore: Refactor cast to use SparkCastOptions param (#1146) * Refactor cast to use SparkCastOptions param * update tests * update benches * update benches * update benches --- .../core/src/execution/datafusion/planner.rs | 23 +-- native/spark-expr/benches/cast_from_string.rs | 30 +--- native/spark-expr/benches/cast_numeric.rs | 22 +-- native/spark-expr/src/cast.rs | 157 +++++++----------- native/spark-expr/src/lib.rs | 4 +- native/spark-expr/src/to_json.rs | 5 +- 6 files changed, 83 insertions(+), 158 deletions(-) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 83f86dbee8..33c4924cbd 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -100,7 +100,8 @@ use datafusion_comet_proto::{ }; use datafusion_comet_spark_expr::{ ArrayInsert, Cast, CreateNamedStruct, DateTruncExpr, GetArrayStructFields, GetStructField, - HourExpr, IfExpr, ListExtract, MinuteExpr, RLike, SecondExpr, TimestampTruncExpr, ToJson, + HourExpr, IfExpr, ListExtract, MinuteExpr, RLike, SecondExpr, SparkCastOptions, + TimestampTruncExpr, ToJson, }; use datafusion_common::scalar::ScalarStructBuilder; use datafusion_common::{ @@ -388,14 +389,11 @@ impl PhysicalPlanner { ExprStruct::Cast(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - let timezone = expr.timezone.clone(); let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; Ok(Arc::new(Cast::new( child, datatype, - eval_mode, - timezone, - expr.allow_incompat, + SparkCastOptions::new(eval_mode, &expr.timezone, expr.allow_incompat), ))) } ExprStruct::Hour(expr) => { @@ -806,24 +804,21 @@ impl PhysicalPlanner { let data_type = return_type.map(to_arrow_datatype).unwrap(); // For some Decimal128 operations, we need wider internal digits. // Cast left and right to Decimal256 and cast the result back to Decimal128 - let left = Arc::new(Cast::new_without_timezone( + let left = Arc::new(Cast::new( left, DataType::Decimal256(p1, s1), - EvalMode::Legacy, - false, + SparkCastOptions::new_without_timezone(EvalMode::Legacy, false), )); - let right = Arc::new(Cast::new_without_timezone( + let right = Arc::new(Cast::new( right, DataType::Decimal256(p2, s2), - EvalMode::Legacy, - false, + SparkCastOptions::new_without_timezone(EvalMode::Legacy, false), )); let child = Arc::new(BinaryExpr::new(left, op, right)); - Ok(Arc::new(Cast::new_without_timezone( + Ok(Arc::new(Cast::new( child, data_type, - EvalMode::Legacy, - false, + SparkCastOptions::new_without_timezone(EvalMode::Legacy, false), ))) } ( diff --git a/native/spark-expr/benches/cast_from_string.rs b/native/spark-expr/benches/cast_from_string.rs index 056ada2eb7..c6b0bcf397 100644 --- a/native/spark-expr/benches/cast_from_string.rs +++ b/native/spark-expr/benches/cast_from_string.rs @@ -18,36 +18,18 @@ use arrow_array::{builder::StringBuilder, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; use criterion::{criterion_group, criterion_main, Criterion}; -use datafusion_comet_spark_expr::{Cast, EvalMode}; +use datafusion_comet_spark_expr::{Cast, EvalMode, SparkCastOptions}; use datafusion_physical_expr::{expressions::Column, PhysicalExpr}; use std::sync::Arc; fn criterion_benchmark(c: &mut Criterion) { let batch = create_utf8_batch(); let expr = Arc::new(Column::new("a", 0)); - let timezone = "".to_string(); - let cast_string_to_i8 = Cast::new( - expr.clone(), - DataType::Int8, - EvalMode::Legacy, - timezone.clone(), - false, - ); - let cast_string_to_i16 = Cast::new( - expr.clone(), - DataType::Int16, - EvalMode::Legacy, - timezone.clone(), - false, - ); - let cast_string_to_i32 = Cast::new( - expr.clone(), - DataType::Int32, - EvalMode::Legacy, - timezone.clone(), - false, - ); - let cast_string_to_i64 = Cast::new(expr, DataType::Int64, EvalMode::Legacy, timezone, false); + let spark_cast_options = SparkCastOptions::new(EvalMode::Legacy, "", false); + let cast_string_to_i8 = Cast::new(expr.clone(), DataType::Int8, spark_cast_options.clone()); + let cast_string_to_i16 = Cast::new(expr.clone(), DataType::Int16, spark_cast_options.clone()); + let cast_string_to_i32 = Cast::new(expr.clone(), DataType::Int32, spark_cast_options.clone()); + let cast_string_to_i64 = Cast::new(expr, DataType::Int64, spark_cast_options); let mut group = c.benchmark_group("cast_string_to_int"); group.bench_function("cast_string_to_i8", |b| { diff --git a/native/spark-expr/benches/cast_numeric.rs b/native/spark-expr/benches/cast_numeric.rs index 15ef1a5a27..8ec8b2f891 100644 --- a/native/spark-expr/benches/cast_numeric.rs +++ b/native/spark-expr/benches/cast_numeric.rs @@ -18,29 +18,17 @@ use arrow_array::{builder::Int32Builder, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; use criterion::{criterion_group, criterion_main, Criterion}; -use datafusion_comet_spark_expr::{Cast, EvalMode}; +use datafusion_comet_spark_expr::{Cast, EvalMode, SparkCastOptions}; use datafusion_physical_expr::{expressions::Column, PhysicalExpr}; use std::sync::Arc; fn criterion_benchmark(c: &mut Criterion) { let batch = create_int32_batch(); let expr = Arc::new(Column::new("a", 0)); - let timezone = "".to_string(); - let cast_i32_to_i8 = Cast::new( - expr.clone(), - DataType::Int8, - EvalMode::Legacy, - timezone.clone(), - false, - ); - let cast_i32_to_i16 = Cast::new( - expr.clone(), - DataType::Int16, - EvalMode::Legacy, - timezone.clone(), - false, - ); - let cast_i32_to_i64 = Cast::new(expr, DataType::Int64, EvalMode::Legacy, timezone, false); + let spark_cast_options = SparkCastOptions::new_without_timezone(EvalMode::Legacy, false); + let cast_i32_to_i8 = Cast::new(expr.clone(), DataType::Int8, spark_cast_options.clone()); + let cast_i32_to_i16 = Cast::new(expr.clone(), DataType::Int16, spark_cast_options.clone()); + let cast_i32_to_i64 = Cast::new(expr, DataType::Int64, spark_cast_options); let mut group = c.benchmark_group("cast_int_to_int"); group.bench_function("cast_i32_to_i8", |b| { diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 13263a5954..f62d0220c9 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -138,14 +138,7 @@ impl TimeStampInfo { pub struct Cast { pub child: Arc, pub data_type: DataType, - pub eval_mode: EvalMode, - - /// When cast from/to timezone related types, we need timezone, which will be resolved with - /// session local timezone by an analyzer in Spark. - pub timezone: String, - - /// Whether to allow casts that are known to be incompatible with Spark - pub allow_incompat: bool, + pub cast_options: SparkCastOptions, } macro_rules! cast_utf8_to_int { @@ -547,30 +540,41 @@ impl Cast { pub fn new( child: Arc, data_type: DataType, - eval_mode: EvalMode, - timezone: String, - allow_incompat: bool, + cast_options: SparkCastOptions, ) -> Self { Self { child, data_type, - timezone, + cast_options, + } + } +} + +/// Spark cast options +#[derive(Debug, Clone, Hash, PartialEq, Eq)] +pub struct SparkCastOptions { + /// Spark evaluation mode + pub eval_mode: EvalMode, + /// When cast from/to timezone related types, we need timezone, which will be resolved with + /// session local timezone by an analyzer in Spark. + pub timezone: String, + /// Allow casts that are supported but not guaranteed to be 100% compatible + pub allow_incompat: bool, +} + +impl SparkCastOptions { + pub fn new(eval_mode: EvalMode, timezone: &str, allow_incompat: bool) -> Self { + Self { eval_mode, + timezone: timezone.to_string(), allow_incompat, } } - pub fn new_without_timezone( - child: Arc, - data_type: DataType, - eval_mode: EvalMode, - allow_incompat: bool, - ) -> Self { + pub fn new_without_timezone(eval_mode: EvalMode, allow_incompat: bool) -> Self { Self { - child, - data_type, - timezone: "".to_string(), eval_mode, + timezone: "".to_string(), allow_incompat, } } @@ -582,33 +586,21 @@ impl Cast { pub fn spark_cast( arg: ColumnarValue, data_type: &DataType, - eval_mode: EvalMode, - timezone: &str, - allow_incompat: bool, + cast_options: &SparkCastOptions, ) -> DataFusionResult { match arg { ColumnarValue::Array(array) => Ok(ColumnarValue::Array(cast_array( array, data_type, - eval_mode, - timezone.to_owned(), - allow_incompat, + cast_options, )?)), ColumnarValue::Scalar(scalar) => { // Note that normally CAST(scalar) should be fold in Spark JVM side. However, for // some cases e.g., scalar subquery, Spark will not fold it, so we need to handle it // here. let array = scalar.to_array()?; - let scalar = ScalarValue::try_from_array( - &cast_array( - array, - data_type, - eval_mode, - timezone.to_owned(), - allow_incompat, - )?, - 0, - )?; + let scalar = + ScalarValue::try_from_array(&cast_array(array, data_type, cast_options)?, 0)?; Ok(ColumnarValue::Scalar(scalar)) } } @@ -617,12 +609,11 @@ pub fn spark_cast( fn cast_array( array: ArrayRef, to_type: &DataType, - eval_mode: EvalMode, - timezone: String, - allow_incompat: bool, + cast_options: &SparkCastOptions, ) -> DataFusionResult { - let array = array_with_timezone(array, timezone.clone(), Some(to_type))?; + let array = array_with_timezone(array, cast_options.timezone.clone(), Some(to_type))?; let from_type = array.data_type().clone(); + let array = match &from_type { DataType::Dictionary(key_type, value_type) if key_type.as_ref() == &DataType::Int32 @@ -636,13 +627,7 @@ fn cast_array( let casted_dictionary = DictionaryArray::::new( dict_array.keys().clone(), - cast_array( - Arc::clone(dict_array.values()), - to_type, - eval_mode, - timezone, - allow_incompat, - )?, + cast_array(Arc::clone(dict_array.values()), to_type, cast_options)?, ); let casted_result = match to_type { @@ -654,6 +639,7 @@ fn cast_array( _ => array, }; let from_type = array.data_type(); + let eval_mode = cast_options.eval_mode; let cast_result = match (from_type, to_type) { (DataType::Utf8, DataType::Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode), @@ -661,7 +647,7 @@ fn cast_array( spark_cast_utf8_to_boolean::(&array, eval_mode) } (DataType::Utf8, DataType::Timestamp(_, _)) => { - cast_string_to_timestamp(&array, to_type, eval_mode, &timezone) + cast_string_to_timestamp(&array, to_type, eval_mode, &cast_options.timezone) } (DataType::Utf8, DataType::Date32) => cast_string_to_date(&array, to_type, eval_mode), (DataType::Int64, DataType::Int32) @@ -712,17 +698,15 @@ fn cast_array( spark_cast_nonintegral_numeric_to_integral(&array, eval_mode, from_type, to_type) } (DataType::Struct(_), DataType::Utf8) => { - Ok(casts_struct_to_string(array.as_struct(), &timezone)?) + Ok(casts_struct_to_string(array.as_struct(), cast_options)?) } (DataType::Struct(_), DataType::Struct(_)) => Ok(cast_struct_to_struct( array.as_struct(), from_type, to_type, - eval_mode, - timezone, - allow_incompat, + cast_options, )?), - _ if is_datafusion_spark_compatible(from_type, to_type, allow_incompat) => { + _ if is_datafusion_spark_compatible(from_type, to_type, cast_options.allow_incompat) => { // use DataFusion cast only when we know that it is compatible with Spark Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) } @@ -825,9 +809,7 @@ fn cast_struct_to_struct( array: &StructArray, from_type: &DataType, to_type: &DataType, - eval_mode: EvalMode, - timezone: String, - allow_incompat: bool, + cast_options: &SparkCastOptions, ) -> DataFusionResult { match (from_type, to_type) { (DataType::Struct(_), DataType::Struct(to_fields)) => { @@ -836,9 +818,7 @@ fn cast_struct_to_struct( let cast_field = cast_array( Arc::clone(array.column(i)), to_fields[i].data_type(), - eval_mode, - timezone.clone(), - allow_incompat, + cast_options, )?; cast_fields.push((Arc::clone(&to_fields[i]), cast_field)); } @@ -848,7 +828,10 @@ fn cast_struct_to_struct( } } -fn casts_struct_to_string(array: &StructArray, timezone: &str) -> DataFusionResult { +fn casts_struct_to_string( + array: &StructArray, + spark_cast_options: &SparkCastOptions, +) -> DataFusionResult { // cast each field to a string let string_arrays: Vec = array .columns() @@ -857,9 +840,7 @@ fn casts_struct_to_string(array: &StructArray, timezone: &str) -> DataFusionResu spark_cast( ColumnarValue::Array(Arc::clone(arr)), &DataType::Utf8, - EvalMode::Legacy, - timezone, - true, + spark_cast_options, ) .and_then(|cv| cv.into_array(arr.len())) }) @@ -1464,7 +1445,7 @@ impl Display for Cast { write!( f, "Cast [data_type: {}, timezone: {}, child: {}, eval_mode: {:?}]", - self.data_type, self.timezone, self.child, &self.eval_mode + self.data_type, self.cast_options.timezone, self.child, &self.cast_options.eval_mode ) } } @@ -1475,9 +1456,8 @@ impl PartialEq for Cast { .downcast_ref::() .map(|x| { self.child.eq(&x.child) - && self.timezone.eq(&x.timezone) + && self.cast_options.eq(&x.cast_options) && self.data_type.eq(&x.data_type) - && self.eval_mode.eq(&x.eval_mode) }) .unwrap_or(false) } @@ -1498,13 +1478,7 @@ impl PhysicalExpr for Cast { fn evaluate(&self, batch: &RecordBatch) -> DataFusionResult { let arg = self.child.evaluate(batch)?; - spark_cast( - arg, - &self.data_type, - self.eval_mode, - &self.timezone, - self.allow_incompat, - ) + spark_cast(arg, &self.data_type, &self.cast_options) } fn children(&self) -> Vec<&Arc> { @@ -1519,9 +1493,7 @@ impl PhysicalExpr for Cast { 1 => Ok(Arc::new(Cast::new( Arc::clone(&children[0]), self.data_type.clone(), - self.eval_mode, - self.timezone.clone(), - self.allow_incompat, + self.cast_options.clone(), ))), _ => internal_err!("Cast should have exactly one child"), } @@ -1531,9 +1503,7 @@ impl PhysicalExpr for Cast { let mut s = state; self.child.hash(&mut s); self.data_type.hash(&mut s); - self.timezone.hash(&mut s); - self.eval_mode.hash(&mut s); - self.allow_incompat.hash(&mut s); + self.cast_options.hash(&mut s); self.hash(&mut s); } } @@ -2110,12 +2080,11 @@ mod tests { let timezone = "UTC".to_string(); // test casting string dictionary array to timestamp array + let cast_options = SparkCastOptions::new(EvalMode::Legacy, &timezone, false); let result = cast_array( dict_array, &DataType::Timestamp(TimeUnit::Microsecond, Some(timezone.clone().into())), - EvalMode::Legacy, - timezone.clone(), - false, + &cast_options, )?; assert_eq!( *result.data_type(), @@ -2320,12 +2289,11 @@ mod tests { fn test_cast_unsupported_timestamp_to_date() { // Since datafusion uses chrono::Datetime internally not all dates representable by TimestampMicrosecondType are supported let timestamps: PrimitiveArray = vec![i64::MAX].into(); + let cast_options = SparkCastOptions::new(EvalMode::Legacy, "UTC", false); let result = cast_array( Arc::new(timestamps.with_timezone("Europe/Copenhagen")), &DataType::Date32, - EvalMode::Legacy, - "UTC".to_owned(), - false, + &cast_options, ); assert!(result.is_err()) } @@ -2333,12 +2301,11 @@ mod tests { #[test] fn test_cast_invalid_timezone() { let timestamps: PrimitiveArray = vec![i64::MAX].into(); + let cast_options = SparkCastOptions::new(EvalMode::Legacy, "Not a valid timezone", false); let result = cast_array( Arc::new(timestamps.with_timezone("Europe/Copenhagen")), &DataType::Date32, - EvalMode::Legacy, - "Not a valid timezone".to_owned(), - false, + &cast_options, ); assert!(result.is_err()) } @@ -2360,9 +2327,7 @@ mod tests { let string_array = cast_array( c, &DataType::Utf8, - EvalMode::Legacy, - "UTC".to_owned(), - false, + &SparkCastOptions::new(EvalMode::Legacy, "UTC", false), ) .unwrap(); let string_array = string_array.as_string::(); @@ -2396,9 +2361,7 @@ mod tests { let cast_array = spark_cast( ColumnarValue::Array(c), &DataType::Struct(fields), - EvalMode::Legacy, - "UTC", - false, + &SparkCastOptions::new(EvalMode::Legacy, "UTC", false), ) .unwrap(); if let ColumnarValue::Array(cast_array) = cast_array { @@ -2429,9 +2392,7 @@ mod tests { let cast_array = spark_cast( ColumnarValue::Array(c), &DataType::Struct(fields), - EvalMode::Legacy, - "UTC", - false, + &SparkCastOptions::new(EvalMode::Legacy, "UTC", false), ) .unwrap(); if let ColumnarValue::Array(cast_array) = cast_array { diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 3ec2e886b5..c227b3a025 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -34,7 +34,7 @@ pub mod timezone; mod to_json; pub mod utils; -pub use cast::{spark_cast, Cast}; +pub use cast::{spark_cast, Cast, SparkCastOptions}; pub use error::{SparkError, SparkResult}; pub use if_expr::IfExpr; pub use list::{ArrayInsert, GetArrayStructFields, ListExtract}; @@ -47,7 +47,7 @@ pub use to_json::ToJson; /// the behavior when processing input values that are invalid or would result in an /// error, such as divide by zero errors, and also affects behavior when converting /// between types. -#[derive(Debug, Hash, PartialEq, Clone, Copy)] +#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] pub enum EvalMode { /// Legacy is the default behavior in Spark prior to Spark 4.0. This mode silently ignores /// or replaces errors during SQL operations. Operations resulting in errors (like diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 7d38cbf1b6..1f68eb8606 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -19,6 +19,7 @@ // of the Spark-specific compatibility features that we need (including // being able to specify Spark-compatible cast from all types to string) +use crate::cast::SparkCastOptions; use crate::{spark_cast, EvalMode}; use arrow_array::builder::StringBuilder; use arrow_array::{Array, ArrayRef, RecordBatch, StringArray, StructArray}; @@ -117,9 +118,7 @@ fn array_to_json_string(arr: &Arc, timezone: &str) -> Result Date: Sat, 7 Dec 2024 13:40:43 -0500 Subject: [PATCH 28/47] Enable more scenarios in CometExecBenchmark. (#1151) --- .../sql/benchmark/CometExecBenchmark.scala | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala index 3dd930f671..3ee37bd668 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala @@ -274,23 +274,23 @@ object CometExecBenchmark extends CometBenchmarkBase { } override def runCometBenchmark(mainArgs: Array[String]): Unit = { -// runBenchmarkWithTable("Subquery", 1024 * 1024 * 10) { v => -// subqueryExecBenchmark(v) -// } -// -// runBenchmarkWithTable("Expand", 1024 * 1024 * 10) { v => -// expandExecBenchmark(v) -// } -// -// runBenchmarkWithTable("Project + Filter", 1024 * 1024 * 10) { v => -// for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { -// numericFilterExecBenchmark(v, fractionOfZeros) -// } -// } -// -// runBenchmarkWithTable("Sort", 1024 * 1024 * 10) { v => -// sortExecBenchmark(v) -// } + runBenchmarkWithTable("Subquery", 1024 * 1024 * 10) { v => + subqueryExecBenchmark(v) + } + + runBenchmarkWithTable("Expand", 1024 * 1024 * 10) { v => + expandExecBenchmark(v) + } + + runBenchmarkWithTable("Project + Filter", 1024 * 1024 * 10) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { + numericFilterExecBenchmark(v, fractionOfZeros) + } + } + + runBenchmarkWithTable("Sort", 1024 * 1024 * 10) { v => + sortExecBenchmark(v) + } runBenchmarkWithTable("BloomFilterAggregate", 1024 * 1024 * 10) { v => for (card <- List(100, 1024, 1024 * 1024)) { From 73f14051adc3bfa513adc54a9af157928472ee0b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 11:45:12 -0700 Subject: [PATCH 29/47] chore: Move more expressions from core crate to spark-expr crate (#1152) * move aggregate expressions to spark-expr crate * move more expressions * move benchmark * normalize_nan * bitwise not * comet scalar funcs * update bench imports --- native/Cargo.lock | 2 + native/Cargo.toml | 1 + native/core/Cargo.toml | 7 +--- native/core/src/common/bit.rs | 6 +-- .../datafusion/expressions/checkoverflow.rs | 15 +------ .../execution/datafusion/expressions/mod.rs | 11 ------ .../core/src/execution/datafusion/planner.rs | 18 +++------ native/core/src/lib.rs | 27 ------------- native/core/src/parquet/read/levels.rs | 7 ++-- native/core/src/parquet/read/values.rs | 2 +- native/spark-expr/Cargo.toml | 8 ++++ .../{core => spark-expr}/benches/aggregate.rs | 6 +-- .../expressions => spark-expr/src}/avg.rs | 0 .../src}/avg_decimal.rs | 2 +- .../src}/bitwise_not.rs | 18 +-------- .../src}/comet_scalar_funcs.rs | 6 +-- .../src}/correlation.rs | 5 +-- .../src}/covariance.rs | 0 native/spark-expr/src/lib.rs | 20 ++++++++++ .../src}/normalize_nan.rs | 0 .../expressions => spark-expr/src}/stddev.rs | 2 +- .../src}/sum_decimal.rs | 10 ++--- native/spark-expr/src/utils.rs | 39 ++++++++++++++++++- .../src}/variance.rs | 0 24 files changed, 96 insertions(+), 116 deletions(-) rename native/{core => spark-expr}/benches/aggregate.rs (97%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/avg.rs (100%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/avg_decimal.rs (99%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/bitwise_not.rs (88%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/comet_scalar_funcs.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/correlation.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/covariance.rs (100%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/normalize_nan.rs (100%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/stddev.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/sum_decimal.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/variance.rs (100%) diff --git a/native/Cargo.lock b/native/Cargo.lock index a7f8359d87..67d041a399 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -942,10 +942,12 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", + "futures", "num", "rand", "regex", "thiserror", + "tokio", "twox-hash 2.0.1", ] diff --git a/native/Cargo.toml b/native/Cargo.toml index 85c46a6d02..4ac85479f2 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -51,6 +51,7 @@ datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.8" } +futures = "0.3.28" num = "0.4" rand = "0.8" regex = "1.9.6" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index daa0837c82..4b9753ec51 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -42,7 +42,7 @@ arrow-data = { workspace = true } arrow-schema = { workspace = true } parquet = { workspace = true, default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } -futures = "0.3.28" +futures = { workspace = true } mimalloc = { version = "*", default-features = false, optional = true } tokio = { version = "1", features = ["rt-multi-thread"] } async-trait = "0.1" @@ -88,7 +88,6 @@ hex = "0.4.3" [features] default = [] -nightly = [] [lib] name = "comet" @@ -123,10 +122,6 @@ harness = false name = "filter" harness = false -[[bench]] -name = "aggregate" -harness = false - [[bench]] name = "bloom_filter_agg" harness = false diff --git a/native/core/src/common/bit.rs b/native/core/src/common/bit.rs index 871786bb15..72d7729d93 100644 --- a/native/core/src/common/bit.rs +++ b/native/core/src/common/bit.rs @@ -17,14 +17,12 @@ use std::{cmp::min, mem::size_of}; -use arrow::buffer::Buffer; - use crate::{ errors::CometResult as Result, - likely, parquet::{data_type::AsBytes, util::bit_packing::unpack32}, - unlikely, }; +use arrow::buffer::Buffer; +use datafusion_comet_spark_expr::utils::{likely, unlikely}; #[inline] pub fn from_ne_slice(bs: &[u8]) -> T { diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs index ed03ab667f..e922171bd2 100644 --- a/native/core/src/execution/datafusion/expressions/checkoverflow.rs +++ b/native/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -27,8 +27,7 @@ use arrow::{ datatypes::{Decimal128Type, DecimalType}, record_batch::RecordBatch, }; -use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; -use arrow_schema::{DataType, Schema, DECIMAL128_MAX_PRECISION}; +use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; @@ -172,15 +171,3 @@ impl PhysicalExpr for CheckOverflow { self.hash(&mut s); } } - -/// Adapted from arrow-rs `validate_decimal_precision` but returns bool -/// instead of Err to avoid the cost of formatting the error strings and is -/// optimized to remove a memcpy that exists in the original function -/// we can remove this code once we upgrade to a version of arrow-rs that -/// includes https://github.com/apache/arrow-rs/pull/6419 -#[inline] -pub fn is_valid_decimal_precision(value: i128, precision: u8) -> bool { - precision <= DECIMAL128_MAX_PRECISION - && value >= MIN_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] - && value <= MAX_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] -} diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 48b80384b0..2bb14df36e 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -17,26 +17,15 @@ //! Native DataFusion expressions -pub mod bitwise_not; pub mod checkoverflow; -mod normalize_nan; -pub use normalize_nan::NormalizeNaNAndZero; use crate::errors::CometError; -pub mod avg; -pub mod avg_decimal; pub mod bloom_filter_agg; pub mod bloom_filter_might_contain; -pub mod comet_scalar_funcs; -pub mod correlation; -pub mod covariance; pub mod negative; -pub mod stddev; pub mod strings; pub mod subquery; -pub mod sum_decimal; pub mod unbound; -pub mod variance; pub use datafusion_comet_spark_expr::{EvalMode, SparkError}; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 33c4924cbd..a83dba5d63 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -18,29 +18,19 @@ //! Converts Spark physical plan to DataFusion physical plan use super::expressions::EvalMode; -use crate::execution::datafusion::expressions::comet_scalar_funcs::create_comet_physical_fun; use crate::execution::operators::{CopyMode, FilterExec}; use crate::{ errors::ExpressionError, execution::{ datafusion::{ expressions::{ - avg::Avg, - avg_decimal::AvgDecimal, - bitwise_not::BitwiseNotExpr, bloom_filter_agg::BloomFilterAgg, bloom_filter_might_contain::BloomFilterMightContain, checkoverflow::CheckOverflow, - correlation::Correlation, - covariance::Covariance, negative, - stddev::Stddev, strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}, subquery::Subquery, - sum_decimal::SumDecimal, unbound::UnboundColumn, - variance::Variance, - NormalizeNaNAndZero, }, operators::expand::CometExpandExec, shuffle_writer::ShuffleWriterExec, @@ -82,6 +72,7 @@ use datafusion::{ }, prelude::SessionContext, }; +use datafusion_comet_spark_expr::create_comet_physical_fun; use datafusion_functions_nested::concat::ArrayAppend; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; @@ -99,9 +90,10 @@ use datafusion_comet_proto::{ spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; use datafusion_comet_spark_expr::{ - ArrayInsert, Cast, CreateNamedStruct, DateTruncExpr, GetArrayStructFields, GetStructField, - HourExpr, IfExpr, ListExtract, MinuteExpr, RLike, SecondExpr, SparkCastOptions, - TimestampTruncExpr, ToJson, + ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Correlation, Covariance, CreateNamedStruct, + DateTruncExpr, GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, + NormalizeNaNAndZero, RLike, SecondExpr, SparkCastOptions, Stddev, SumDecimal, + TimestampTruncExpr, ToJson, Variance, }; use datafusion_common::scalar::ScalarStructBuilder; use datafusion_common::{ diff --git a/native/core/src/lib.rs b/native/core/src/lib.rs index c6a7a4143d..68c8ae7299 100644 --- a/native/core/src/lib.rs +++ b/native/core/src/lib.rs @@ -104,30 +104,3 @@ fn default_logger_config() -> CometResult { .build(root) .map_err(|err| CometError::Config(err.to_string())) } - -// These are borrowed from hashbrown crate: -// https://github.com/rust-lang/hashbrown/blob/master/src/raw/mod.rs - -// On stable we can use #[cold] to get a equivalent effect: this attributes -// suggests that the function is unlikely to be called -#[cfg(not(feature = "nightly"))] -#[inline] -#[cold] -fn cold() {} - -#[cfg(not(feature = "nightly"))] -#[inline] -fn likely(b: bool) -> bool { - if !b { - cold(); - } - b -} -#[cfg(not(feature = "nightly"))] -#[inline] -fn unlikely(b: bool) -> bool { - if b { - cold(); - } - b -} diff --git a/native/core/src/parquet/read/levels.rs b/native/core/src/parquet/read/levels.rs index 3d74b277cc..9077c0e485 100644 --- a/native/core/src/parquet/read/levels.rs +++ b/native/core/src/parquet/read/levels.rs @@ -17,15 +17,14 @@ use std::mem; -use arrow::buffer::Buffer; -use parquet::schema::types::ColumnDescPtr; - use super::values::Decoder; use crate::{ common::bit::{self, read_u32, BitReader}, parquet::ParquetMutableVector, - unlikely, }; +use arrow::buffer::Buffer; +use datafusion_comet_spark_expr::utils::unlikely; +use parquet::schema::types::ColumnDescPtr; const INITIAL_BUF_LEN: usize = 16; diff --git a/native/core/src/parquet/read/values.rs b/native/core/src/parquet/read/values.rs index b439e29e6a..71cd035d2e 100644 --- a/native/core/src/parquet/read/values.rs +++ b/native/core/src/parquet/read/values.rs @@ -28,9 +28,9 @@ use crate::write_val_or_null; use crate::{ common::bit::{self, BitReader}, parquet::{data_type::*, ParquetMutableVector}, - unlikely, }; use arrow::datatypes::DataType as ArrowDataType; +use datafusion_comet_spark_expr::utils::unlikely; pub fn get_decoder( value_data: Buffer, diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 532bf74375..65517431d2 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -29,6 +29,7 @@ edition = { workspace = true } [dependencies] arrow = { workspace = true } arrow-array = { workspace = true } +arrow-data = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true } @@ -39,12 +40,14 @@ chrono-tz = { workspace = true } num = { workspace = true } regex = { workspace = true } thiserror = { workspace = true } +futures = { workspace = true } twox-hash = "2.0.0" [dev-dependencies] arrow-data = {workspace = true} criterion = "0.5.1" rand = { workspace = true} +tokio = { version = "1", features = ["rt-multi-thread"] } [lib] @@ -66,3 +69,8 @@ harness = false [[bench]] name = "decimal_div" harness = false + +[[bench]] +name = "aggregate" +harness = false + diff --git a/native/core/benches/aggregate.rs b/native/spark-expr/benches/aggregate.rs similarity index 97% rename from native/core/benches/aggregate.rs rename to native/spark-expr/benches/aggregate.rs index c6209406fd..43194fdda2 100644 --- a/native/core/benches/aggregate.rs +++ b/native/spark-expr/benches/aggregate.rs @@ -19,16 +19,16 @@ use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::builder::{Decimal128Builder, StringBuilder}; use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; -use comet::execution::datafusion::expressions::avg_decimal::AvgDecimal; -use comet::execution::datafusion::expressions::sum_decimal::SumDecimal; use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion::execution::TaskContext; use datafusion::functions_aggregate::average::avg_udaf; use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; -use datafusion_execution::TaskContext; +use datafusion_comet_spark_expr::AvgDecimal; +use datafusion_comet_spark_expr::SumDecimal; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Column; diff --git a/native/core/src/execution/datafusion/expressions/avg.rs b/native/spark-expr/src/avg.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/avg.rs rename to native/spark-expr/src/avg.rs diff --git a/native/core/src/execution/datafusion/expressions/avg_decimal.rs b/native/spark-expr/src/avg_decimal.rs similarity index 99% rename from native/core/src/execution/datafusion/expressions/avg_decimal.rs rename to native/spark-expr/src/avg_decimal.rs index a265fdc29e..163e1560b6 100644 --- a/native/core/src/execution/datafusion/expressions/avg_decimal.rs +++ b/native/spark-expr/src/avg_decimal.rs @@ -28,7 +28,7 @@ use datafusion_common::{not_impl_err, Result, ScalarValue}; use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::expressions::checkoverflow::is_valid_decimal_precision; +use crate::utils::is_valid_decimal_precision; use arrow_array::ArrowNativeTypeOp; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use datafusion::logical_expr::Volatility::Immutable; diff --git a/native/core/src/execution/datafusion/expressions/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs similarity index 88% rename from native/core/src/execution/datafusion/expressions/bitwise_not.rs rename to native/spark-expr/src/bitwise_not.rs index a2b9ebe5b5..36234935e1 100644 --- a/native/core/src/execution/datafusion/expressions/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -28,7 +28,7 @@ use arrow::{ }; use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; macro_rules! compute_op { @@ -135,22 +135,6 @@ pub fn bitwise_not(arg: Arc) -> Result> Ok(Arc::new(BitwiseNotExpr::new(arg))) } -fn scalar_bitwise_not(scalar: ScalarValue) -> Result { - match scalar { - ScalarValue::Int8(None) - | ScalarValue::Int16(None) - | ScalarValue::Int32(None) - | ScalarValue::Int64(None) => Ok(scalar), - ScalarValue::Int8(Some(v)) => Ok(ScalarValue::Int8(Some(!v))), - ScalarValue::Int16(Some(v)) => Ok(ScalarValue::Int16(Some(!v))), - ScalarValue::Int32(Some(v)) => Ok(ScalarValue::Int32(Some(!v))), - ScalarValue::Int64(Some(v)) => Ok(ScalarValue::Int64(Some(!v))), - value => Err(DataFusionError::Internal(format!( - "Can not run ! on scalar value {value:?}" - ))), - } -} - #[cfg(test)] mod tests { use arrow::datatypes::*; diff --git a/native/core/src/execution/datafusion/expressions/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/comet_scalar_funcs.rs rename to native/spark-expr/src/comet_scalar_funcs.rs index 06717aabeb..71ff0e9dcc 100644 --- a/native/core/src/execution/datafusion/expressions/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -15,15 +15,15 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::DataType; -use datafusion_comet_spark_expr::scalar_funcs::hash_expressions::{ +use crate::scalar_funcs::hash_expressions::{ spark_sha224, spark_sha256, spark_sha384, spark_sha512, }; -use datafusion_comet_spark_expr::scalar_funcs::{ +use crate::scalar_funcs::{ spark_ceil, spark_date_add, spark_date_sub, spark_decimal_div, spark_floor, spark_hex, spark_isnan, spark_make_decimal, spark_murmur3_hash, spark_read_side_padding, spark_round, spark_unhex, spark_unscaled_value, spark_xxhash64, SparkChrFunc, }; +use arrow_schema::DataType; use datafusion_common::{DataFusionError, Result as DataFusionResult}; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{ diff --git a/native/core/src/execution/datafusion/expressions/correlation.rs b/native/spark-expr/src/correlation.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/correlation.rs rename to native/spark-expr/src/correlation.rs index 6bf35e7115..e5f36c6f95 100644 --- a/native/core/src/execution/datafusion/expressions/correlation.rs +++ b/native/spark-expr/src/correlation.rs @@ -19,9 +19,8 @@ use arrow::compute::{and, filter, is_not_null}; use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::expressions::{ - covariance::CovarianceAccumulator, stddev::StddevAccumulator, -}; +use crate::covariance::CovarianceAccumulator; +use crate::stddev::StddevAccumulator; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, diff --git a/native/core/src/execution/datafusion/expressions/covariance.rs b/native/spark-expr/src/covariance.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/covariance.rs rename to native/spark-expr/src/covariance.rs diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index c227b3a025..15f446ef35 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -23,18 +23,38 @@ mod cast; mod error; mod if_expr; +mod avg; +pub use avg::Avg; +mod bitwise_not; +pub use bitwise_not::{bitwise_not, BitwiseNotExpr}; +mod avg_decimal; +pub use avg_decimal::AvgDecimal; +mod correlation; +pub use correlation::Correlation; +mod covariance; +pub use covariance::Covariance; mod kernels; mod list; mod regexp; pub mod scalar_funcs; pub mod spark_hash; +mod stddev; +pub use stddev::Stddev; mod structs; +mod sum_decimal; +pub use sum_decimal::SumDecimal; +mod normalize_nan; mod temporal; pub mod timezone; mod to_json; pub mod utils; +pub use normalize_nan::NormalizeNaNAndZero; +mod variance; +pub use variance::Variance; +mod comet_scalar_funcs; pub use cast::{spark_cast, Cast, SparkCastOptions}; +pub use comet_scalar_funcs::create_comet_physical_fun; pub use error::{SparkError, SparkResult}; pub use if_expr::IfExpr; pub use list::{ArrayInsert, GetArrayStructFields, ListExtract}; diff --git a/native/core/src/execution/datafusion/expressions/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/normalize_nan.rs rename to native/spark-expr/src/normalize_nan.rs diff --git a/native/core/src/execution/datafusion/expressions/stddev.rs b/native/spark-expr/src/stddev.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/stddev.rs rename to native/spark-expr/src/stddev.rs index 1ba495e215..3cf604da0b 100644 --- a/native/core/src/execution/datafusion/expressions/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -17,7 +17,7 @@ use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::expressions::variance::VarianceAccumulator; +use crate::variance::VarianceAccumulator; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, diff --git a/native/core/src/execution/datafusion/expressions/sum_decimal.rs b/native/spark-expr/src/sum_decimal.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/sum_decimal.rs rename to native/spark-expr/src/sum_decimal.rs index d885ff90b6..ab142aee69 100644 --- a/native/core/src/execution/datafusion/expressions/sum_decimal.rs +++ b/native/spark-expr/src/sum_decimal.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::datafusion::expressions::checkoverflow::is_valid_decimal_precision; -use crate::unlikely; +use crate::utils::{is_valid_decimal_precision, unlikely}; use arrow::{ array::BooleanBufferBuilder, buffer::{BooleanBuffer, NullBuffer}, @@ -113,7 +112,6 @@ impl AggregateUDFImpl for SumDecimal { Ok(Box::new(SumDecimalGroupsAccumulator::new( self.result_type.clone(), self.precision, - self.scale, ))) } @@ -286,18 +284,16 @@ struct SumDecimalGroupsAccumulator { sum: Vec, result_type: DataType, precision: u8, - scale: i8, } impl SumDecimalGroupsAccumulator { - fn new(result_type: DataType, precision: u8, scale: i8) -> Self { + fn new(result_type: DataType, precision: u8) -> Self { Self { is_not_null: BooleanBufferBuilder::new(0), is_empty: BooleanBufferBuilder::new(0), sum: Vec::new(), result_type, precision, - scale, } } @@ -488,11 +484,11 @@ mod tests { use arrow::datatypes::*; use arrow_array::builder::{Decimal128Builder, StringBuilder}; use arrow_array::RecordBatch; + use datafusion::execution::TaskContext; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; use datafusion_common::Result; - use datafusion_execution::TaskContext; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::{Column, Literal}; diff --git a/native/spark-expr/src/utils.rs b/native/spark-expr/src/utils.rs index db4ad1956a..18a2314fb1 100644 --- a/native/spark-expr/src/utils.rs +++ b/native/spark-expr/src/utils.rs @@ -19,7 +19,7 @@ use arrow_array::{ cast::as_primitive_array, types::{Int32Type, TimestampMicrosecondType}, }; -use arrow_schema::{ArrowError, DataType}; +use arrow_schema::{ArrowError, DataType, DECIMAL128_MAX_PRECISION}; use std::sync::Arc; use crate::timezone::Tz; @@ -27,6 +27,7 @@ use arrow::{ array::{as_dictionary_array, Array, ArrayRef, PrimitiveArray}, temporal_conversions::as_datetime, }; +use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use chrono::{DateTime, Offset, TimeZone}; /// Preprocesses input arrays to add timezone information from Spark to Arrow array datatype or @@ -176,3 +177,39 @@ fn pre_timestamp_cast(array: ArrayRef, timezone: String) -> Result Ok(array), } } + +/// Adapted from arrow-rs `validate_decimal_precision` but returns bool +/// instead of Err to avoid the cost of formatting the error strings and is +/// optimized to remove a memcpy that exists in the original function +/// we can remove this code once we upgrade to a version of arrow-rs that +/// includes https://github.com/apache/arrow-rs/pull/6419 +#[inline] +pub fn is_valid_decimal_precision(value: i128, precision: u8) -> bool { + precision <= DECIMAL128_MAX_PRECISION + && value >= MIN_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] + && value <= MAX_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] +} + +// These are borrowed from hashbrown crate: +// https://github.com/rust-lang/hashbrown/blob/master/src/raw/mod.rs + +// On stable we can use #[cold] to get a equivalent effect: this attributes +// suggests that the function is unlikely to be called +#[inline] +#[cold] +pub fn cold() {} + +#[inline] +pub fn likely(b: bool) -> bool { + if !b { + cold(); + } + b +} +#[inline] +pub fn unlikely(b: bool) -> bool { + if b { + cold(); + } + b +} diff --git a/native/core/src/execution/datafusion/expressions/variance.rs b/native/spark-expr/src/variance.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/variance.rs rename to native/spark-expr/src/variance.rs From 5c45fdc9e311da07a0f3510d9b8aec673081861a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 17:45:27 -0700 Subject: [PATCH 30/47] remove dead code (#1155) --- native/core/benches/bloom_filter_agg.rs | 2 - native/core/benches/parquet_read.rs | 5 +- native/core/src/errors.rs | 44 +------ .../expressions/bloom_filter_agg.rs | 6 - .../core/src/execution/datafusion/planner.rs | 3 - .../datafusion/util/spark_bit_array.rs | 1 + native/core/src/execution/jni_api.rs | 8 -- native/core/src/execution/kernels/strings.rs | 117 +----------------- native/core/src/execution/operators/scan.rs | 8 -- native/core/src/execution/shuffle/list.rs | 4 +- native/core/src/execution/shuffle/map.rs | 17 +-- native/core/src/execution/shuffle/row.rs | 5 +- native/core/src/execution/utils.rs | 18 --- native/core/src/jvm_bridge/batch_iterator.rs | 1 + .../core/src/jvm_bridge/comet_metric_node.rs | 1 + .../jvm_bridge/comet_task_memory_manager.rs | 1 + native/core/src/jvm_bridge/mod.rs | 1 + native/core/src/lib.rs | 1 - native/core/src/parquet/mod.rs | 2 - native/core/src/parquet/mutable_vector.rs | 8 -- native/core/src/parquet/read/column.rs | 14 +-- native/core/src/parquet/read/mod.rs | 3 - native/core/src/parquet/read/values.rs | 26 +--- .../src/parquet/util/test_common/page_util.rs | 12 +- 24 files changed, 27 insertions(+), 281 deletions(-) diff --git a/native/core/benches/bloom_filter_agg.rs b/native/core/benches/bloom_filter_agg.rs index af3eb919ee..25d27d174e 100644 --- a/native/core/benches/bloom_filter_agg.rs +++ b/native/core/benches/bloom_filter_agg.rs @@ -61,10 +61,8 @@ fn criterion_benchmark(c: &mut Criterion) { group.bench_function(agg_mode.0, |b| { let comet_bloom_filter_agg = Arc::new(AggregateUDF::new_from_impl(BloomFilterAgg::new( - Arc::clone(&c0), Arc::clone(&num_items), Arc::clone(&num_bits), - "bloom_filter_agg", DataType::Binary, ))); b.to_async(&rt).iter(|| { diff --git a/native/core/benches/parquet_read.rs b/native/core/benches/parquet_read.rs index 1f8178cd22..ae511ade51 100644 --- a/native/core/benches/parquet_read.rs +++ b/native/core/benches/parquet_read.rs @@ -44,7 +44,7 @@ fn bench(c: &mut Criterion) { let mut group = c.benchmark_group("comet_parquet_read"); let schema = build_test_schema(); - let pages = build_plain_int32_pages(schema.clone(), schema.column(0), 0.0); + let pages = build_plain_int32_pages(schema.column(0), 0.0); group.bench_function("INT/PLAIN/NOT_NULL", |b| { let t = TypePtr::new( PrimitiveTypeBuilder::new("f", PhysicalType::INT32) @@ -107,7 +107,6 @@ const VALUES_PER_PAGE: usize = 10_000; const BATCH_SIZE: usize = 4096; fn build_plain_int32_pages( - schema: SchemaDescPtr, column_desc: ColumnDescPtr, null_density: f32, ) -> impl PageIterator + Clone { @@ -143,7 +142,7 @@ fn build_plain_int32_pages( // Since `InMemoryPageReader` is not exposed from parquet crate, here we use // `InMemoryPageIterator` instead which is a Iter>. - InMemoryPageIterator::new(schema, column_desc, vec![pages]) + InMemoryPageIterator::new(vec![pages]) } struct TestColumnReader { diff --git a/native/core/src/errors.rs b/native/core/src/errors.rs index 92799bcf6f..4d623d976f 100644 --- a/native/core/src/errors.rs +++ b/native/core/src/errors.rs @@ -485,23 +485,6 @@ where || f(t) } -// This is a duplicate of `try_unwrap_or_throw`, which is used to work around Arrow's lack of -// `UnwindSafe` handling. -pub fn try_assert_unwind_safe_or_throw(env: &JNIEnv, f: F) -> T -where - T: JNIDefault, - F: FnOnce(JNIEnv) -> Result, -{ - let mut env1 = unsafe { JNIEnv::from_raw(env.get_raw()).unwrap() }; - let env2 = unsafe { JNIEnv::from_raw(env.get_raw()).unwrap() }; - unwrap_or_throw_default( - &mut env1, - flatten( - catch_unwind(std::panic::AssertUnwindSafe(curry(f, env2))).map_err(CometError::from), - ), - ) -} - // It is currently undefined behavior to unwind from Rust code into foreign code, so we can wrap // our JNI functions and turn these panics into a `RuntimeException`. pub fn try_unwrap_or_throw(env: &JNIEnv, f: F) -> T @@ -534,10 +517,7 @@ mod tests { AttachGuard, InitArgsBuilder, JNIEnv, JNIVersion, JavaVM, }; - use assertables::{ - assert_contains, assert_contains_as_result, assert_starts_with, - assert_starts_with_as_result, - }; + use assertables::{assert_starts_with, assert_starts_with_as_result}; pub fn jvm() -> &'static Arc { static mut JVM: Option> = None; @@ -890,26 +870,4 @@ mod tests { // first line. assert_starts_with!(msg_rust, expected_message); } - - // Asserts that exception's message matches `expected_message`. - fn assert_exception_message_with_stacktrace( - env: &mut JNIEnv, - exception: JThrowable, - expected_message: &str, - stacktrace_contains: &str, - ) { - let message = env - .call_method(exception, "getMessage", "()Ljava/lang/String;", &[]) - .unwrap() - .l() - .unwrap(); - let message_string = message.into(); - let msg_rust: String = env.get_string(&message_string).unwrap().into(); - // Since panics result in multi-line messages which include the backtrace, just use the - // first line. - assert_starts_with!(msg_rust, expected_message); - - // Check that the stacktrace is included by checking for a specific element - assert_contains!(msg_rust, stacktrace_contains); - } } diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs index e6528a5633..1300e08c22 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs +++ b/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs @@ -34,9 +34,7 @@ use datafusion_physical_expr::expressions::Literal; #[derive(Debug, Clone)] pub struct BloomFilterAgg { - name: String, signature: Signature, - expr: Arc, num_items: i32, num_bits: i32, } @@ -53,15 +51,12 @@ fn extract_i32_from_literal(expr: Arc) -> i32 { impl BloomFilterAgg { pub fn new( - expr: Arc, num_items: Arc, num_bits: Arc, - name: impl Into, data_type: DataType, ) -> Self { assert!(matches!(data_type, DataType::Binary)); Self { - name: name.into(), signature: Signature::uniform( 1, vec![ @@ -73,7 +68,6 @@ impl BloomFilterAgg { ], Volatility::Immutable, ), - expr, num_items: extract_i32_from_literal(num_items), num_bits: extract_i32_from_literal(num_bits), } diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index a83dba5d63..5e77b3f653 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -115,7 +115,6 @@ use std::cmp::max; use std::{collections::HashMap, sync::Arc}; // For clippy error on type_complexity. -type ExecResult = Result; type PhyAggResult = Result, ExecutionError>; type PhyExprResult = Result, String)>, ExecutionError>; type PartitionPhyExprResult = Result>, ExecutionError>; @@ -1758,10 +1757,8 @@ impl PhysicalPlanner { self.create_expr(expr.num_bits.as_ref().unwrap(), Arc::clone(&schema))?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); let func = AggregateUDF::new_from_impl(BloomFilterAgg::new( - Arc::clone(&child), Arc::clone(&num_items), Arc::clone(&num_bits), - "bloom_filter_agg", datatype, )); Self::create_aggr_func_expr("bloom_filter_agg", schema, vec![child], func) diff --git a/native/core/src/execution/datafusion/util/spark_bit_array.rs b/native/core/src/execution/datafusion/util/spark_bit_array.rs index 68b97d6608..6cfecc1bfb 100644 --- a/native/core/src/execution/datafusion/util/spark_bit_array.rs +++ b/native/core/src/execution/datafusion/util/spark_bit_array.rs @@ -70,6 +70,7 @@ impl SparkBitArray { self.data.len() } + #[allow(dead_code)] // this is only called from tests pub fn cardinality(&self) -> usize { self.bit_count } diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 8afe134cd3..5103f5ce4a 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -207,14 +207,6 @@ fn prepare_datafusion_session_context( Ok(session_ctx) } -fn parse_bool(conf: &HashMap, name: &str) -> CometResult { - conf.get(name) - .map(String::as_str) - .unwrap_or("false") - .parse::() - .map_err(|e| CometError::Config(format!("Failed to parse boolean config {name}: {e}"))) -} - /// Prepares arrow arrays for output. fn prepare_output( env: &mut JNIEnv, diff --git a/native/core/src/execution/kernels/strings.rs b/native/core/src/execution/kernels/strings.rs index 2e5e67b67f..d63b2c4778 100644 --- a/native/core/src/execution/kernels/strings.rs +++ b/native/core/src/execution/kernels/strings.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use arrow::{ array::*, - buffer::{Buffer, MutableBuffer}, + buffer::MutableBuffer, compute::kernels::substring::{substring as arrow_substring, substring_by_char}, datatypes::{DataType, Int32Type}, }; @@ -87,43 +87,6 @@ pub fn substring(array: &dyn Array, start: i64, length: u64) -> Result ArrayRef { - match array.data_type() { - DataType::LargeUtf8 => generic_substring( - array - .as_any() - .downcast_ref::() - .expect("A large string is expected"), - start, - length, - |i| i as i64, - ), - DataType::Utf8 => generic_substring( - array - .as_any() - .downcast_ref::() - .expect("A string is expected"), - start, - length, - |i| i, - ), - _ => panic!("substring does not support type {:?}", array.data_type()), - } -} - fn generic_string_space(length: &Int32Array) -> ArrayRef { let array_len = length.len(); let mut offsets = MutableBuffer::new((array_len + 1) * std::mem::size_of::()); @@ -163,81 +126,3 @@ fn generic_string_space(length: &Int32Array) -> Arr }; make_array(data) } - -fn generic_substring( - array: &GenericStringArray, - start: &Int32Array, - length: &Int32Array, - f: F, -) -> ArrayRef -where - F: Fn(i32) -> OffsetSize, -{ - assert_eq!(array.len(), start.len()); - assert_eq!(array.len(), length.len()); - - // compute current offsets - let offsets = array.to_data().buffers()[0].clone(); - let offsets: &[OffsetSize] = offsets.typed_data::(); - - // compute null bitmap (copy) - let null_bit_buffer = array.to_data().nulls().map(|b| b.buffer().clone()); - - // Gets slices of start and length arrays to access them directly for performance. - let start_data = start.to_data(); - let length_data = length.to_data(); - let starts = start_data.buffers()[0].typed_data::(); - let lengths = length_data.buffers()[0].typed_data::(); - - // compute values - let array_data = array.to_data(); - let values = &array_data.buffers()[1]; - let data = values.as_slice(); - - // we have no way to estimate how much this will be. - let mut new_values = MutableBuffer::new(0); - let mut new_offsets: Vec = Vec::with_capacity(array.len() + 1); - - let mut length_so_far = OffsetSize::zero(); - new_offsets.push(length_so_far); - (0..array.len()).for_each(|i| { - // the length of this entry - let length_i: OffsetSize = offsets[i + 1] - offsets[i]; - // compute where we should start slicing this entry - let start_pos: OffsetSize = f(starts[i]); - - let start = offsets[i] - + if start_pos >= OffsetSize::zero() { - start_pos - } else { - length_i + start_pos - }; - - let start = start.clamp(offsets[i], offsets[i + 1]); - // compute the length of the slice - let slice_length: OffsetSize = f(lengths[i].max(0)).min(offsets[i + 1] - start); - - length_so_far += slice_length; - - new_offsets.push(length_so_far); - - // we need usize for ranges - let start = start.to_usize().unwrap(); - let slice_length = slice_length.to_usize().unwrap(); - - new_values.extend_from_slice(&data[start..start + slice_length]); - }); - - let data = unsafe { - ArrayData::new_unchecked( - GenericStringArray::::DATA_TYPE, - array.len(), - None, - null_bit_buffer, - 0, - vec![Buffer::from_slice_ref(&new_offsets), new_values.into()], - vec![], - ) - }; - make_array(data) -} diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index a97caf0db1..0d35859dfb 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -525,12 +525,4 @@ impl InputBatch { InputBatch::Batch(columns, num_rows) } - - /// Get the number of rows in this batch - fn num_rows(&self) -> usize { - match self { - Self::EOF => 0, - Self::Batch(_, num_rows) => *num_rows, - } - } } diff --git a/native/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs index d8bdcb197b..0f7f3e8cb3 100644 --- a/native/core/src/execution/shuffle/list.rs +++ b/native/core/src/execution/shuffle/list.rs @@ -28,7 +28,6 @@ use arrow_schema::{DataType, TimeUnit}; pub struct SparkUnsafeArray { row_addr: i64, - row_size: i32, num_elements: usize, element_offset: i64, } @@ -45,7 +44,7 @@ impl SparkUnsafeObject for SparkUnsafeArray { impl SparkUnsafeArray { /// Creates a `SparkUnsafeArray` which points to the given address and size in bytes. - pub fn new(addr: i64, size: i32) -> Self { + pub fn new(addr: i64) -> Self { // Read the number of elements from the first 8 bytes. let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr as *const u8, 8) }; let num_elements = i64::from_le_bytes(slice.try_into().unwrap()); @@ -60,7 +59,6 @@ impl SparkUnsafeArray { Self { row_addr: addr, - row_size: size, num_elements: num_elements as usize, element_offset: addr + Self::get_header_portion_in_bytes(num_elements), } diff --git a/native/core/src/execution/shuffle/map.rs b/native/core/src/execution/shuffle/map.rs index 014695293a..0969168f8e 100644 --- a/native/core/src/execution/shuffle/map.rs +++ b/native/core/src/execution/shuffle/map.rs @@ -30,8 +30,6 @@ use arrow_array::builder::{ use arrow_schema::{DataType, FieldRef, Fields, TimeUnit}; pub struct SparkUnsafeMap { - row_addr: i64, - row_size: i32, pub(crate) keys: SparkUnsafeArray, pub(crate) values: SparkUnsafeArray, } @@ -59,8 +57,8 @@ impl SparkUnsafeMap { panic!("Negative value size in bytes of map: {}", value_array_size); } - let keys = SparkUnsafeArray::new(addr + 8, key_array_size as i32); - let values = SparkUnsafeArray::new(addr + 8 + key_array_size, value_array_size); + let keys = SparkUnsafeArray::new(addr + 8); + let values = SparkUnsafeArray::new(addr + 8 + key_array_size); if keys.get_num_elements() != values.get_num_elements() { panic!( @@ -70,16 +68,7 @@ impl SparkUnsafeMap { ); } - Self { - row_addr: addr, - row_size: size, - keys, - values, - } - } - - pub(crate) fn get_num_elements(&self) -> usize { - self.keys.get_num_elements() + Self { keys, values } } } diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 2aeb488154..17b180e9d0 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -48,7 +48,6 @@ use std::{ sync::Arc, }; -const WORD_SIZE: i64 = 8; const MAX_LONG_DIGITS: u8 = 18; const NESTED_TYPE_BUILDER_CAPACITY: usize = 100; @@ -170,8 +169,8 @@ pub trait SparkUnsafeObject { /// Returns array value at the given index of the object. fn get_array(&self, index: usize) -> SparkUnsafeArray { - let (offset, len) = self.get_offset_and_len(index); - SparkUnsafeArray::new(self.get_row_addr() + offset as i64, len) + let (offset, _) = self.get_offset_and_len(index); + SparkUnsafeArray::new(self.get_row_addr() + offset as i64) } fn get_map(&self, index: usize) -> SparkUnsafeMap { diff --git a/native/core/src/execution/utils.rs b/native/core/src/execution/utils.rs index 553d426062..4992b7ba94 100644 --- a/native/core/src/execution/utils.rs +++ b/native/core/src/execution/utils.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - use arrow::{ array::ArrayData, error::ArrowError, @@ -52,10 +50,6 @@ pub trait SparkArrowConvert { where Self: Sized; - /// Convert Arrow Arrays to C data interface. - /// It returns a tuple (ArrowArray address, ArrowSchema address). - fn to_spark(&self) -> Result<(i64, i64), ExecutionError>; - /// Move Arrow Arrays to C data interface. fn move_to_spark(&self, array: i64, schema: i64) -> Result<(), ExecutionError>; } @@ -88,18 +82,6 @@ impl SparkArrowConvert for ArrayData { Ok(ffi_array) } - /// Converts this ArrowData to pointers of Arrow C data interface. - /// Returned pointers are Arc-ed and should be freed manually. - #[allow(clippy::arc_with_non_send_sync)] - fn to_spark(&self) -> Result<(i64, i64), ExecutionError> { - let arrow_array = Arc::new(FFI_ArrowArray::new(self)); - let arrow_schema = Arc::new(FFI_ArrowSchema::try_from(self.data_type())?); - - let (array, schema) = (Arc::into_raw(arrow_array), Arc::into_raw(arrow_schema)); - - Ok((array as i64, schema as i64)) - } - /// Move this ArrowData to pointers of Arrow C data interface. fn move_to_spark(&self, array: i64, schema: i64) -> Result<(), ExecutionError> { let array_ptr = array as *mut FFI_ArrowArray; diff --git a/native/core/src/jvm_bridge/batch_iterator.rs b/native/core/src/jvm_bridge/batch_iterator.rs index 45b10cf208..998e540c73 100644 --- a/native/core/src/jvm_bridge/batch_iterator.rs +++ b/native/core/src/jvm_bridge/batch_iterator.rs @@ -24,6 +24,7 @@ use jni::{ }; /// A struct that holds all the JNI methods and fields for JVM `CometBatchIterator` class. +#[allow(dead_code)] // we need to keep references to Java items to prevent GC pub struct CometBatchIterator<'a> { pub class: JClass<'a>, pub method_has_next: JMethodID, diff --git a/native/core/src/jvm_bridge/comet_metric_node.rs b/native/core/src/jvm_bridge/comet_metric_node.rs index 8647e071a2..85386d9b0d 100644 --- a/native/core/src/jvm_bridge/comet_metric_node.rs +++ b/native/core/src/jvm_bridge/comet_metric_node.rs @@ -23,6 +23,7 @@ use jni::{ }; /// A struct that holds all the JNI methods and fields for JVM CometMetricNode class. +#[allow(dead_code)] // we need to keep references to Java items to prevent GC pub struct CometMetricNode<'a> { pub class: JClass<'a>, pub method_get_child_node: JMethodID, diff --git a/native/core/src/jvm_bridge/comet_task_memory_manager.rs b/native/core/src/jvm_bridge/comet_task_memory_manager.rs index 97d1bf3a73..22c3332c61 100644 --- a/native/core/src/jvm_bridge/comet_task_memory_manager.rs +++ b/native/core/src/jvm_bridge/comet_task_memory_manager.rs @@ -25,6 +25,7 @@ use jni::{ /// A wrapper which delegate acquire/release memory calls to the /// JVM side `CometTaskMemoryManager`. #[derive(Debug)] +#[allow(dead_code)] // we need to keep references to Java items to prevent GC pub struct CometTaskMemoryManager<'a> { pub class: JClass<'a>, pub method_acquire_memory: JMethodID, diff --git a/native/core/src/jvm_bridge/mod.rs b/native/core/src/jvm_bridge/mod.rs index 4936b1c5bf..5fc0a55e3e 100644 --- a/native/core/src/jvm_bridge/mod.rs +++ b/native/core/src/jvm_bridge/mod.rs @@ -189,6 +189,7 @@ pub use comet_metric_node::*; pub use comet_task_memory_manager::*; /// The JVM classes that are used in the JNI calls. +#[allow(dead_code)] // we need to keep references to Java items to prevent GC pub struct JVMClasses<'a> { /// Cached JClass for "java.lang.Object" java_lang_object: JClass<'a>, diff --git a/native/core/src/lib.rs b/native/core/src/lib.rs index 68c8ae7299..cab511faff 100644 --- a/native/core/src/lib.rs +++ b/native/core/src/lib.rs @@ -17,7 +17,6 @@ #![allow(incomplete_features)] #![allow(non_camel_case_types)] -#![allow(dead_code)] #![allow(clippy::upper_case_acronyms)] // For prost generated struct #![allow(clippy::derive_partial_eq_without_eq)] diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs index 455f19929f..d2a6f4804b 100644 --- a/native/core/src/parquet/mod.rs +++ b/native/core/src/parquet/mod.rs @@ -47,8 +47,6 @@ use util::jni::{convert_column_descriptor, convert_encoding}; use self::util::jni::TypePromotionInfo; -const STR_CLASS_NAME: &str = "java/lang/String"; - /// Parquet read context maintained across multiple JNI calls. struct Context { pub column_reader: ColumnReader, diff --git a/native/core/src/parquet/mutable_vector.rs b/native/core/src/parquet/mutable_vector.rs index 7f30d7d877..d19ea32fad 100644 --- a/native/core/src/parquet/mutable_vector.rs +++ b/native/core/src/parquet/mutable_vector.rs @@ -40,12 +40,6 @@ pub struct ParquetMutableVector { /// The number of null elements in this vector, must <= `num_values`. pub(crate) num_nulls: usize, - /// The capacity of the vector - pub(crate) capacity: usize, - - /// How many bits are required to store a single value - pub(crate) bit_width: usize, - /// The validity buffer of this Arrow vector. A bit set at position `i` indicates the `i`th /// element is not null. Otherwise, an unset bit at position `i` indicates the `i`th element is /// null. @@ -109,8 +103,6 @@ impl ParquetMutableVector { arrow_type, num_values: 0, num_nulls: 0, - capacity, - bit_width, validity_buffer, value_buffer, children, diff --git a/native/core/src/parquet/read/column.rs b/native/core/src/parquet/read/column.rs index 73f8df9560..05a0bf7b5b 100644 --- a/native/core/src/parquet/read/column.rs +++ b/native/core/src/parquet/read/column.rs @@ -770,7 +770,7 @@ impl TypedColumnReader { // Create a new vector for dictionary values let mut value_vector = ParquetMutableVector::new(page_value_count, &self.arrow_type); - let mut dictionary = self.get_decoder(page_data, page_value_count, encoding); + let mut dictionary = self.get_decoder(page_data, encoding); dictionary.read_batch(&mut value_vector, page_value_count); value_vector.num_values = page_value_count; @@ -812,7 +812,7 @@ impl TypedColumnReader { self.def_level_decoder = Some(dl_decoder); page_buffer = page_buffer.slice(offset); - let value_decoder = self.get_decoder(page_buffer, page_value_count, encoding); + let value_decoder = self.get_decoder(page_buffer, encoding); self.value_decoder = Some(value_decoder); } @@ -838,7 +838,7 @@ impl TypedColumnReader { dl_decoder.set_data(page_value_count, &def_level_data); self.def_level_decoder = Some(dl_decoder); - let value_decoder = self.get_decoder(value_data, page_value_count, encoding); + let value_decoder = self.get_decoder(value_data, encoding); self.value_decoder = Some(value_decoder); } @@ -977,15 +977,9 @@ impl TypedColumnReader { } } - fn get_decoder( - &self, - value_data: Buffer, - page_value_count: usize, - encoding: Encoding, - ) -> Box { + fn get_decoder(&self, value_data: Buffer, encoding: Encoding) -> Box { get_decoder::( value_data, - page_value_count, encoding, Arc::clone(&self.desc), self.read_options, diff --git a/native/core/src/parquet/read/mod.rs b/native/core/src/parquet/read/mod.rs index 4d057a06c9..5a55f21170 100644 --- a/native/core/src/parquet/read/mod.rs +++ b/native/core/src/parquet/read/mod.rs @@ -44,9 +44,6 @@ pub struct PlainDecoderInner { /// The current offset in `data`, in bytes. offset: usize, - /// The number of total values in `data` - value_count: usize, - /// Reads `data` bit by bit, used if `T` is [`BoolType`]. bit_reader: BitReader, diff --git a/native/core/src/parquet/read/values.rs b/native/core/src/parquet/read/values.rs index 71cd035d2e..e28d695ecd 100644 --- a/native/core/src/parquet/read/values.rs +++ b/native/core/src/parquet/read/values.rs @@ -34,20 +34,16 @@ use datafusion_comet_spark_expr::utils::unlikely; pub fn get_decoder( value_data: Buffer, - num_values: usize, encoding: Encoding, desc: ColumnDescPtr, read_options: ReadOptions, ) -> Box { let decoder: Box = match encoding { - Encoding::PLAIN | Encoding::PLAIN_DICTIONARY => Box::new(PlainDecoder::::new( - value_data, - num_values, - desc, - read_options, - )), + Encoding::PLAIN | Encoding::PLAIN_DICTIONARY => { + Box::new(PlainDecoder::::new(value_data, desc, read_options)) + } // This is for dictionary indices - Encoding::RLE_DICTIONARY => Box::new(DictDecoder::new(value_data, num_values)), + Encoding::RLE_DICTIONARY => Box::new(DictDecoder::new(value_data)), _ => panic!("Unsupported encoding: {}", encoding), }; decoder @@ -108,17 +104,11 @@ pub struct PlainDecoder { } impl PlainDecoder { - pub fn new( - value_data: Buffer, - num_values: usize, - desc: ColumnDescPtr, - read_options: ReadOptions, - ) -> Self { + pub fn new(value_data: Buffer, desc: ColumnDescPtr, read_options: ReadOptions) -> Self { let len = value_data.len(); let inner = PlainDecoderInner { data: value_data.clone(), offset: 0, - value_count: num_values, bit_reader: BitReader::new(value_data, len), read_options, desc, @@ -938,9 +928,6 @@ pub struct DictDecoder { /// Number of bits used to represent dictionary indices. Must be between `[0, 64]`. bit_width: usize, - /// The number of total values in `data` - value_count: usize, - /// Bit reader bit_reader: BitReader, @@ -955,12 +942,11 @@ pub struct DictDecoder { } impl DictDecoder { - pub fn new(buf: Buffer, num_values: usize) -> Self { + pub fn new(buf: Buffer) -> Self { let bit_width = buf.as_bytes()[0] as usize; Self { bit_width, - value_count: num_values, bit_reader: BitReader::new_all(buf.slice(1)), rle_left: 0, bit_packed_left: 0, diff --git a/native/core/src/parquet/util/test_common/page_util.rs b/native/core/src/parquet/util/test_common/page_util.rs index e20cc30cff..333298bc37 100644 --- a/native/core/src/parquet/util/test_common/page_util.rs +++ b/native/core/src/parquet/util/test_common/page_util.rs @@ -28,7 +28,7 @@ use parquet::{ levels::{max_buffer_size, LevelEncoder}, }, errors::Result, - schema::types::{ColumnDescPtr, SchemaDescPtr}, + schema::types::ColumnDescPtr, }; use super::random_numbers_range; @@ -201,20 +201,12 @@ impl + Send> Iterator for InMemoryPageReader

{ /// A utility page iterator which stores page readers in memory, used for tests. #[derive(Clone)] pub struct InMemoryPageIterator>> { - schema: SchemaDescPtr, - column_desc: ColumnDescPtr, page_reader_iter: I, } impl>> InMemoryPageIterator { - pub fn new( - schema: SchemaDescPtr, - column_desc: ColumnDescPtr, - pages: impl IntoIterator, IntoIter = I>, - ) -> Self { + pub fn new(pages: impl IntoIterator, IntoIter = I>) -> Self { Self { - schema, - column_desc, page_reader_iter: pages.into_iter(), } } From 2c1a6b9c576042f2e0a73cf76a3ad93db46b6a0b Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA Date: Wed, 11 Dec 2024 09:44:12 -0800 Subject: [PATCH 31/47] fix: Spark 4.0-preview1 SPARK-47120 (#1156) ## Which issue does this PR close? Part of https://github.com/apache/datafusion-comet/issues/372 and https://github.com/apache/datafusion-comet/issues/551 ## Rationale for this change To be ready for Spark 4.0 ## What changes are included in this PR? This PR fixes the new test SPARK-47120 added in Spark 4.0 ## How are these changes tested? tests enabled --- dev/diffs/4.0.0-preview1.diff | 16 +++------------- docs/source/user-guide/source.md | 2 +- .../apache/comet/parquet/ParquetFilters.scala | 10 ++++++---- 3 files changed, 10 insertions(+), 18 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 1a6810a7ab..ba68d2a7bd 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -2088,7 +2088,7 @@ index cd6f41b4ef4..4b6a17344bc 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 795e9f46a8d..6285a1e388b 100644 +index 795e9f46a8d..5306c94a686 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1100,7 +1100,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared @@ -2150,17 +2150,7 @@ index 795e9f46a8d..6285a1e388b 100644 // block 1: // null count min max // page-0 0 0 99 -@@ -2211,7 +2225,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared - } - } - -- test("SPARK-47120: subquery literal filter pushdown") { -+ test("SPARK-47120: subquery literal filter pushdown", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { - withTable("t1", "t2") { - sql("create table t1(d date) using parquet") - sql("create table t2(d date) using parquet") -@@ -2301,7 +2316,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { +@@ -2301,7 +2315,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2173,7 +2163,7 @@ index 795e9f46a8d..6285a1e388b 100644 } else { assert(selectedFilters.isEmpty, "There is filter pushed down") } -@@ -2362,7 +2381,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { +@@ -2362,7 +2380,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") diff --git a/docs/source/user-guide/source.md b/docs/source/user-guide/source.md index 71c9060cb5..5f5d03d906 100644 --- a/docs/source/user-guide/source.md +++ b/docs/source/user-guide/source.md @@ -27,7 +27,7 @@ Official source releases can be downloaded from https://dist.apache.org/repos/di ```console # Pick the latest version -export COMET_VERSION=0.3.0 +export COMET_VERSION=0.4.0 # Download the tarball curl -O "https://dist.apache.org/repos/dist/release/datafusion/datafusion-comet-$COMET_VERSION/apache-datafusion-comet-$COMET_VERSION.tar.gz" # Unpack diff --git a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala index 17844aba82..bcb23986f3 100644 --- a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala +++ b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala @@ -723,20 +723,22 @@ class ParquetFilters( .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.LessThan(name, value) if canMakeFilterOn(name, value) => + case sources.LessThan(name, value) if (value != null) && canMakeFilterOn(name, value) => makeLt .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.LessThanOrEqual(name, value) if canMakeFilterOn(name, value) => + case sources.LessThanOrEqual(name, value) + if (value != null) && canMakeFilterOn(name, value) => makeLtEq .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.GreaterThan(name, value) if canMakeFilterOn(name, value) => + case sources.GreaterThan(name, value) if (value != null) && canMakeFilterOn(name, value) => makeGt .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) - case sources.GreaterThanOrEqual(name, value) if canMakeFilterOn(name, value) => + case sources.GreaterThanOrEqual(name, value) + if (value != null) && canMakeFilterOn(name, value) => makeGtEq .lift(nameToParquetField(name).fieldType) .map(_(nameToParquetField(name).fieldNames, value)) From 49cf0d7f32813b6cb5bd3999378c960bd1171fca Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 12 Dec 2024 07:45:06 -0700 Subject: [PATCH 32/47] chore: Move string kernels and expressions to spark-expr crate (#1164) * Move string kernels and expressions to spark-expr crate * remove unused hash kernel * remove unused dependencies --- native/Cargo.lock | 2 - native/core/Cargo.toml | 6 - native/core/benches/hash.rs | 137 ------------- .../execution/datafusion/expressions/mod.rs | 1 - .../core/src/execution/datafusion/planner.rs | 15 +- native/core/src/execution/kernels/hash.rs | 187 ------------------ native/core/src/execution/kernels/mod.rs | 23 --- native/core/src/execution/mod.rs | 3 - native/spark-expr/src/kernels/mod.rs | 1 + .../src}/kernels/strings.rs | 7 +- native/spark-expr/src/lib.rs | 2 + .../expressions => spark-expr/src}/strings.rs | 2 +- 12 files changed, 13 insertions(+), 373 deletions(-) delete mode 100644 native/core/benches/hash.rs delete mode 100644 native/core/src/execution/kernels/hash.rs delete mode 100644 native/core/src/execution/kernels/mod.rs rename native/{core/src/execution => spark-expr/src}/kernels/strings.rs (96%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/strings.rs (99%) diff --git a/native/Cargo.lock b/native/Cargo.lock index 67d041a399..9a8eab83e6 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -871,7 +871,6 @@ dependencies = [ name = "datafusion-comet" version = "0.5.0" dependencies = [ - "ahash", "arrow", "arrow-array", "arrow-buffer", @@ -893,7 +892,6 @@ dependencies = [ "datafusion-physical-expr", "flate2", "futures", - "half", "hex", "itertools 0.11.0", "jni", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 4b9753ec51..489da46d47 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -41,7 +41,6 @@ arrow-buffer = { workspace = true } arrow-data = { workspace = true } arrow-schema = { workspace = true } parquet = { workspace = true, default-features = false, features = ["experimental"] } -half = { version = "2.4.1", default-features = false } futures = { workspace = true } mimalloc = { version = "*", default-features = false, optional = true } tokio = { version = "1", features = ["rt-multi-thread"] } @@ -62,7 +61,6 @@ rand = { workspace = true} num = { workspace = true } bytes = "1.5.0" tempfile = "3.8.0" -ahash = { version = "0.8", default-features = false } itertools = "0.11.0" paste = "1.0.14" datafusion-common = { workspace = true } @@ -102,10 +100,6 @@ harness = false name = "bit_util" harness = false -[[bench]] -name = "hash" -harness = false - [[bench]] name = "row_columnar" harness = false diff --git a/native/core/benches/hash.rs b/native/core/benches/hash.rs deleted file mode 100644 index 039a6d5d8e..0000000000 --- a/native/core/benches/hash.rs +++ /dev/null @@ -1,137 +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. - -#[path = "common.rs"] -mod common; - -use arrow_array::ArrayRef; -use comet::execution::kernels::hash; -use common::*; -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use datafusion_comet_spark_expr::scalar_funcs::{spark_murmur3_hash, spark_xxhash64}; -use datafusion_common::ScalarValue; -use datafusion_expr::ColumnarValue; -use std::sync::Arc; - -const BATCH_SIZE: usize = 1024 * 8; -const NUM_ITER: usize = 10; -const NULL_FRACTION: f32 = 0.1; - -fn criterion_benchmark(c: &mut Criterion) { - let mut group = c.benchmark_group("hash"); - - let a1: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); - let a2: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); - let a3: ArrayRef = Arc::new(create_int64_array( - BATCH_SIZE, - NULL_FRACTION, - 0, - BATCH_SIZE as i64, - )); - let a4: ArrayRef = Arc::new(create_int64_array( - BATCH_SIZE, - NULL_FRACTION, - 0, - BATCH_SIZE as i64, - )); - - group.bench_function( - BenchmarkId::new("hash_i64_single_nonnull", BATCH_SIZE), - |b| { - let input = vec![a1.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }, - ); - group.bench_function(BenchmarkId::new("hash_i64_single_null", BATCH_SIZE), |b| { - let input = vec![a3.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }); - group.bench_function( - BenchmarkId::new("hash_i64_multiple_nonnull", BATCH_SIZE), - |b| { - let input = vec![a1.clone(), a2.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }, - ); - group.bench_function( - BenchmarkId::new("hash_i64_multiple_null", BATCH_SIZE), - |b| { - let input = vec![a3.clone(), a4.clone()]; - let mut dst = vec![0; BATCH_SIZE]; - - b.iter(|| { - for _ in 0..NUM_ITER { - hash(&input, &mut dst); - } - }); - }, - ); - group.bench_function(BenchmarkId::new("xxhash64", BATCH_SIZE), |b| { - let inputs = &[ - ColumnarValue::Array(a3.clone()), - ColumnarValue::Array(a4.clone()), - ColumnarValue::Scalar(ScalarValue::Int64(Some(42i64))), - ]; - - b.iter(|| { - for _ in 0..NUM_ITER { - spark_xxhash64(inputs).unwrap(); - } - }); - }); - group.bench_function(BenchmarkId::new("murmur3", BATCH_SIZE), |b| { - let inputs = &[ - ColumnarValue::Array(a3.clone()), - ColumnarValue::Array(a4.clone()), - ColumnarValue::Scalar(ScalarValue::Int32(Some(42))), - ]; - b.iter(|| { - for _ in 0..NUM_ITER { - spark_murmur3_hash(inputs).unwrap(); - } - }); - }); -} - -fn config() -> Criterion { - Criterion::default() -} - -criterion_group! { - name = benches; - config = config(); - targets = criterion_benchmark -} -criterion_main!(benches); diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 2bb14df36e..5f9f322b2e 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -23,7 +23,6 @@ use crate::errors::CometError; pub mod bloom_filter_agg; pub mod bloom_filter_might_contain; pub mod negative; -pub mod strings; pub mod subquery; pub mod unbound; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 5e77b3f653..0e64ed6afa 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -25,12 +25,8 @@ use crate::{ datafusion::{ expressions::{ bloom_filter_agg::BloomFilterAgg, - bloom_filter_might_contain::BloomFilterMightContain, - checkoverflow::CheckOverflow, - negative, - strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}, - subquery::Subquery, - unbound::UnboundColumn, + bloom_filter_might_contain::BloomFilterMightContain, checkoverflow::CheckOverflow, + negative, subquery::Subquery, unbound::UnboundColumn, }, operators::expand::CometExpandExec, shuffle_writer::ShuffleWriterExec, @@ -90,9 +86,10 @@ use datafusion_comet_proto::{ spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; use datafusion_comet_spark_expr::{ - ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Correlation, Covariance, CreateNamedStruct, - DateTruncExpr, GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, - NormalizeNaNAndZero, RLike, SecondExpr, SparkCastOptions, Stddev, SumDecimal, + ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Contains, Correlation, Covariance, + CreateNamedStruct, DateTruncExpr, EndsWith, GetArrayStructFields, GetStructField, HourExpr, + IfExpr, Like, ListExtract, MinuteExpr, NormalizeNaNAndZero, RLike, SecondExpr, + SparkCastOptions, StartsWith, Stddev, StringSpaceExpr, SubstringExpr, SumDecimal, TimestampTruncExpr, ToJson, Variance, }; use datafusion_common::scalar::ScalarStructBuilder; diff --git a/native/core/src/execution/kernels/hash.rs b/native/core/src/execution/kernels/hash.rs deleted file mode 100644 index b39fd62243..0000000000 --- a/native/core/src/execution/kernels/hash.rs +++ /dev/null @@ -1,187 +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. - -use crate::common::bit; -use ahash::RandomState; -use arrow::datatypes::{i256, ArrowNativeType}; -use arrow_array::{ - downcast_dictionary_array, downcast_primitive_array, Array, ArrayAccessor, ArrayRef, - ArrowPrimitiveType, PrimitiveArray, -}; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; -use std::fmt::Debug; - -pub fn hash(src: &[ArrayRef], dst: &mut [u64]) { - let state = RandomState::with_seed(42); - src.iter().enumerate().for_each(|(idx, v)| { - downcast_dictionary_array!( - v => { - let keys = v.keys(); - let values = v.values(); - downcast_primitive_array!( - values => hash_dict_typed(&state, idx > 0, keys, values, dst), - dt => panic!("Expected only primitive type but found {}", dt) - ) - }, - dt => { - downcast_primitive_array!( - v => hash_typed(&state, idx > 0, v, dst), - _ => panic!("Expected only primitive type but found {}", dt) - ) - } - ) - }); -} - -fn hash_typed(state: &RandomState, mix: bool, array: T, dst: &mut [u64]) -where - T: ArrayAccessor, - T::Item: Hashable + Debug, -{ - let nullable = array.null_count() > 0; - let num_values = array.len(); - if nullable { - for i in 0..num_values { - if !array.is_null(i) { - unsafe { - let value = array.value_unchecked(i); - hash1(state, mix, i, value, dst); - } - } - } - } else { - for i in 0..num_values { - unsafe { - let value = array.value_unchecked(i); - hash1(state, mix, i, value, dst); - } - } - } -} - -fn hash_dict_typed( - state: &RandomState, - mix: bool, - keys: &PrimitiveArray, - values: V, - dst: &mut [u64], -) where - K: ArrowPrimitiveType, - V: ArrayAccessor, - V::Item: Hashable + Debug, -{ - let nullable = keys.null_count() > 0; - let num_keys = keys.len(); - let mut value_hashes = vec![0; values.len()]; - - for (i, value_hash) in value_hashes.iter_mut().enumerate() { - unsafe { - *value_hash = values.value_unchecked(i).create_hash(state); - } - } - - if nullable { - for i in 0..num_keys { - if !keys.is_null(i) { - unsafe { - let idx = keys.value_unchecked(i); - let hash = value_hashes[idx.as_usize()]; - hash1_helper(mix, i, hash, dst); - } - } - } - } else { - for i in 0..num_keys { - unsafe { - let idx = keys.value_unchecked(i); - let hash = value_hashes[idx.as_usize()]; - hash1_helper(mix, i, hash, dst); - } - } - } -} - -#[inline(always)] -fn hash1(state: &RandomState, mix: bool, i: usize, value: T, dst: &mut [u64]) { - let hash = value.create_hash(state); - hash1_helper(mix, i, hash, dst); -} - -#[inline(always)] -fn hash1_helper(mix: bool, i: usize, hash: u64, dst: &mut [u64]) { - dst[i] = if mix { - bit::mix_hash(dst[i], hash) - } else { - hash - } -} - -pub(crate) trait Hashable { - fn create_hash(&self, state: &RandomState) -> u64; -} - -macro_rules! impl_hashable { - ($($t:ty),+) => { - $(impl Hashable for $t { - #[inline] - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self) - } - })+ - }; -} - -impl_hashable!(i8, i16, i32, u8, u16, u32, u64, i128, i256); - -impl Hashable for i64 { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self) - } -} - -impl Hashable for half::f16 { - fn create_hash(&self, _: &RandomState) -> u64 { - unimplemented!("hashing on f16 is not supported") - } -} - -impl Hashable for f32 { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(u32::from_ne_bytes(self.to_ne_bytes())) - } -} - -impl Hashable for f64 { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(u64::from_ne_bytes(self.to_ne_bytes())) - } -} - -impl Hashable for IntervalDayTime { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self.days); - state.hash_one(self.milliseconds) - } -} - -impl Hashable for IntervalMonthDayNano { - fn create_hash(&self, state: &RandomState) -> u64 { - state.hash_one(self.months); - state.hash_one(self.days); - state.hash_one(self.nanoseconds) - } -} diff --git a/native/core/src/execution/kernels/mod.rs b/native/core/src/execution/kernels/mod.rs deleted file mode 100644 index 675dcd4893..0000000000 --- a/native/core/src/execution/kernels/mod.rs +++ /dev/null @@ -1,23 +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. - -//! Kernels - -mod hash; -pub use hash::hash; - -pub(crate) mod strings; diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index f17935702c..3dba747f2d 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -18,9 +18,6 @@ //! PoC of vectorization execution through JNI to Rust. pub mod datafusion; pub mod jni_api; - -pub mod kernels; // for benchmarking - mod metrics; pub mod operators; pub mod serde; diff --git a/native/spark-expr/src/kernels/mod.rs b/native/spark-expr/src/kernels/mod.rs index 88aa34b1a3..3669ff13ad 100644 --- a/native/spark-expr/src/kernels/mod.rs +++ b/native/spark-expr/src/kernels/mod.rs @@ -17,4 +17,5 @@ //! Kernels +pub mod strings; pub(crate) mod temporal; diff --git a/native/core/src/execution/kernels/strings.rs b/native/spark-expr/src/kernels/strings.rs similarity index 96% rename from native/core/src/execution/kernels/strings.rs rename to native/spark-expr/src/kernels/strings.rs index d63b2c4778..bb275fbb9f 100644 --- a/native/core/src/execution/kernels/strings.rs +++ b/native/spark-expr/src/kernels/strings.rs @@ -25,15 +25,14 @@ use arrow::{ compute::kernels::substring::{substring as arrow_substring, substring_by_char}, datatypes::{DataType, Int32Type}, }; - -use crate::errors::ExpressionError; +use datafusion_common::DataFusionError; /// Returns an ArrayRef with a string consisting of `length` spaces. /// /// # Preconditions /// /// - elements in `length` must not be negative -pub fn string_space(length: &dyn Array) -> Result { +pub fn string_space(length: &dyn Array) -> Result { match length.data_type() { DataType::Int32 => { let array = length.as_any().downcast_ref::().unwrap(); @@ -52,7 +51,7 @@ pub fn string_space(length: &dyn Array) -> Result { } } -pub fn substring(array: &dyn Array, start: i64, length: u64) -> Result { +pub fn substring(array: &dyn Array, start: i64, length: u64) -> Result { match array.data_type() { DataType::LargeUtf8 => substring_by_char( array diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 15f446ef35..5dff6e0b8f 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -33,6 +33,8 @@ mod correlation; pub use correlation::Correlation; mod covariance; pub use covariance::Covariance; +mod strings; +pub use strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}; mod kernels; mod list; mod regexp; diff --git a/native/core/src/execution/datafusion/expressions/strings.rs b/native/spark-expr/src/strings.rs similarity index 99% rename from native/core/src/execution/datafusion/expressions/strings.rs rename to native/spark-expr/src/strings.rs index 200b4ec5a3..a8aab6aee9 100644 --- a/native/core/src/execution/datafusion/expressions/strings.rs +++ b/native/spark-expr/src/strings.rs @@ -17,7 +17,7 @@ #![allow(deprecated)] -use crate::execution::kernels::strings::{string_space, substring}; +use crate::kernels::strings::{string_space, substring}; use arrow::{ compute::{ contains_dyn, contains_utf8_scalar_dyn, ends_with_dyn, ends_with_utf8_scalar_dyn, like_dyn, From 7db9aa67ed0049bb25e783bc4ba991e53df34bbf Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 12 Dec 2024 14:51:19 -0700 Subject: [PATCH 33/47] chore: Move remaining expressions to spark-expr crate + some minor refactoring (#1165) * move CheckOverflow to spark-expr crate * move NegativeExpr to spark-expr crate * move UnboundColumn to spark-expr crate * move ExpandExec from execution::datafusion::operators to execution::operators * refactoring to remove datafusion subpackage * update imports in benches * fix * fix --- native/Cargo.lock | 1 + native/core/benches/bloom_filter_agg.rs | 2 +- native/core/benches/shuffle_writer.rs | 2 +- .../execution/datafusion/expressions/mod.rs | 35 ------------------- .../src/execution/datafusion/operators/mod.rs | 18 ---------- .../expressions/bloom_filter_agg.rs | 4 +-- .../expressions/bloom_filter_might_contain.rs | 4 +-- .../{datafusion => expressions}/mod.rs | 13 ++++--- .../{datafusion => }/expressions/subquery.rs | 0 native/core/src/execution/jni_api.rs | 6 ++-- native/core/src/execution/metrics/utils.rs | 2 +- native/core/src/execution/mod.rs | 5 ++- .../{datafusion => }/operators/expand.rs | 10 +++--- native/core/src/execution/operators/mod.rs | 2 ++ native/core/src/execution/operators/scan.rs | 3 +- .../src/execution/{datafusion => }/planner.rs | 32 ++++++++--------- native/core/src/execution/shuffle/mod.rs | 2 ++ native/core/src/execution/shuffle/row.rs | 2 +- .../{datafusion => shuffle}/shuffle_writer.rs | 0 .../execution/{datafusion => }/spark_plan.rs | 0 .../execution/{datafusion => }/util/mod.rs | 0 .../{datafusion => }/util/spark_bit_array.rs | 0 .../util/spark_bloom_filter.rs | 4 +-- native/spark-expr/Cargo.toml | 1 + .../src}/checkoverflow.rs | 0 native/spark-expr/src/lib.rs | 12 +++++++ .../src}/negative.rs | 7 ++-- .../expressions => spark-expr/src}/unbound.rs | 0 28 files changed, 63 insertions(+), 104 deletions(-) delete mode 100644 native/core/src/execution/datafusion/expressions/mod.rs delete mode 100644 native/core/src/execution/datafusion/operators/mod.rs rename native/core/src/execution/{datafusion => }/expressions/bloom_filter_agg.rs (97%) rename native/core/src/execution/{datafusion => }/expressions/bloom_filter_might_contain.rs (97%) rename native/core/src/execution/{datafusion => expressions}/mod.rs (83%) rename native/core/src/execution/{datafusion => }/expressions/subquery.rs (100%) rename native/core/src/execution/{datafusion => }/operators/expand.rs (97%) rename native/core/src/execution/{datafusion => }/planner.rs (98%) rename native/core/src/execution/{datafusion => shuffle}/shuffle_writer.rs (100%) rename native/core/src/execution/{datafusion => }/spark_plan.rs (100%) rename native/core/src/execution/{datafusion => }/util/mod.rs (100%) rename native/core/src/execution/{datafusion => }/util/spark_bit_array.rs (100%) rename native/core/src/execution/{datafusion => }/util/spark_bloom_filter.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/checkoverflow.rs (100%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/negative.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/unbound.rs (100%) diff --git a/native/Cargo.lock b/native/Cargo.lock index 9a8eab83e6..7966bb80bb 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -931,6 +931,7 @@ version = "0.5.0" dependencies = [ "arrow", "arrow-array", + "arrow-buffer", "arrow-data", "arrow-schema", "chrono", diff --git a/native/core/benches/bloom_filter_agg.rs b/native/core/benches/bloom_filter_agg.rs index 25d27d174e..b83ff3fad1 100644 --- a/native/core/benches/bloom_filter_agg.rs +++ b/native/core/benches/bloom_filter_agg.rs @@ -19,7 +19,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::builder::Int64Builder; use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; -use comet::execution::datafusion::expressions::bloom_filter_agg::BloomFilterAgg; +use comet::execution::expressions::bloom_filter_agg::BloomFilterAgg; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 6f28718618..272887238e 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -17,7 +17,7 @@ use arrow_array::{builder::StringBuilder, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; -use comet::execution::datafusion::shuffle_writer::ShuffleWriterExec; +use comet::execution::shuffle::ShuffleWriterExec; use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::{ physical_plan::{common::collect, memory::MemoryExec, ExecutionPlan}, diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs deleted file mode 100644 index 5f9f322b2e..0000000000 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ /dev/null @@ -1,35 +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. - -//! Native DataFusion expressions - -pub mod checkoverflow; - -use crate::errors::CometError; -pub mod bloom_filter_agg; -pub mod bloom_filter_might_contain; -pub mod negative; -pub mod subquery; -pub mod unbound; - -pub use datafusion_comet_spark_expr::{EvalMode, SparkError}; - -fn arithmetic_overflow_error(from_type: &str) -> CometError { - CometError::Spark(SparkError::ArithmeticOverflow { - from_type: from_type.to_string(), - }) -} diff --git a/native/core/src/execution/datafusion/operators/mod.rs b/native/core/src/execution/datafusion/operators/mod.rs deleted file mode 100644 index 3d28a266a6..0000000000 --- a/native/core/src/execution/datafusion/operators/mod.rs +++ /dev/null @@ -1,18 +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. - -pub mod expand; diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs b/native/core/src/execution/expressions/bloom_filter_agg.rs similarity index 97% rename from native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs rename to native/core/src/execution/expressions/bloom_filter_agg.rs index 1300e08c22..ea8bb3647f 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_agg.rs +++ b/native/core/src/execution/expressions/bloom_filter_agg.rs @@ -19,8 +19,8 @@ use arrow_schema::Field; use datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::util::spark_bloom_filter; -use crate::execution::datafusion::util::spark_bloom_filter::SparkBloomFilter; +use crate::execution::util::spark_bloom_filter; +use crate::execution::util::spark_bloom_filter::SparkBloomFilter; use arrow::array::ArrayRef; use arrow_array::BinaryArray; use datafusion::error::Result; diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/expressions/bloom_filter_might_contain.rs similarity index 97% rename from native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs rename to native/core/src/execution/expressions/bloom_filter_might_contain.rs index de922d8312..af6a5a47a3 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/expressions/bloom_filter_might_contain.rs @@ -15,9 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::{ - execution::datafusion::util::spark_bloom_filter::SparkBloomFilter, parquet::data_type::AsBytes, -}; +use crate::{execution::util::spark_bloom_filter::SparkBloomFilter, parquet::data_type::AsBytes}; use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; diff --git a/native/core/src/execution/datafusion/mod.rs b/native/core/src/execution/expressions/mod.rs similarity index 83% rename from native/core/src/execution/datafusion/mod.rs rename to native/core/src/execution/expressions/mod.rs index ca41fa0aa0..e2f811fa2e 100644 --- a/native/core/src/execution/datafusion/mod.rs +++ b/native/core/src/execution/expressions/mod.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -//! Native execution through DataFusion +//! Native DataFusion expressions -pub mod expressions; -mod operators; -pub mod planner; -pub mod shuffle_writer; -pub(crate) mod spark_plan; -mod util; +pub mod bloom_filter_agg; +pub mod bloom_filter_might_contain; +pub mod subquery; + +pub use datafusion_comet_spark_expr::EvalMode; diff --git a/native/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/expressions/subquery.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/subquery.rs rename to native/core/src/execution/expressions/subquery.rs diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 5103f5ce4a..491b389c99 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -45,8 +45,8 @@ use super::{serde, utils::SparkArrowConvert, CometMemoryPool}; use crate::{ errors::{try_unwrap_or_throw, CometError, CometResult}, execution::{ - datafusion::planner::PhysicalPlanner, metrics::utils::update_comet_metric, - serde::to_arrow_datatype, shuffle::row::process_sorted_row_partition, sort::RdxSort, + metrics::utils::update_comet_metric, planner::PhysicalPlanner, serde::to_arrow_datatype, + shuffle::row::process_sorted_row_partition, sort::RdxSort, }, jvm_bridge::{jni_new_global_ref, JVMClasses}, }; @@ -59,8 +59,8 @@ use jni::{ }; use tokio::runtime::Runtime; -use crate::execution::datafusion::spark_plan::SparkPlan; use crate::execution::operators::ScanExec; +use crate::execution::spark_plan::SparkPlan; use log::info; /// Comet native execution context. Kept alive across JNI calls. diff --git a/native/core/src/execution/metrics/utils.rs b/native/core/src/execution/metrics/utils.rs index 4bb1c4474c..0eb4b631dd 100644 --- a/native/core/src/execution/metrics/utils.rs +++ b/native/core/src/execution/metrics/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::datafusion::spark_plan::SparkPlan; +use crate::execution::spark_plan::SparkPlan; use crate::jvm_bridge::jni_new_global_ref; use crate::{ errors::CometError, diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index 3dba747f2d..a74ec3017e 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -16,13 +16,16 @@ // under the License. //! PoC of vectorization execution through JNI to Rust. -pub mod datafusion; +pub mod expressions; pub mod jni_api; mod metrics; pub mod operators; +pub(crate) mod planner; pub mod serde; pub mod shuffle; pub(crate) mod sort; +pub(crate) mod spark_plan; +pub(crate) mod util; pub use datafusion_comet_spark_expr::timezone; pub(crate) mod utils; diff --git a/native/core/src/execution/datafusion/operators/expand.rs b/native/core/src/execution/operators/expand.rs similarity index 97% rename from native/core/src/execution/datafusion/operators/expand.rs rename to native/core/src/execution/operators/expand.rs index a3dd06507e..fb43a6e49f 100644 --- a/native/core/src/execution/datafusion/operators/expand.rs +++ b/native/core/src/execution/operators/expand.rs @@ -37,14 +37,14 @@ use std::{ /// A Comet native operator that expands a single row into multiple rows. This behaves as same as /// Spark Expand operator. #[derive(Debug)] -pub struct CometExpandExec { +pub struct ExpandExec { projections: Vec>>, child: Arc, schema: SchemaRef, cache: PlanProperties, } -impl CometExpandExec { +impl ExpandExec { /// Create a new ExpandExec pub fn new( projections: Vec>>, @@ -66,7 +66,7 @@ impl CometExpandExec { } } -impl DisplayAs for CometExpandExec { +impl DisplayAs for ExpandExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { @@ -87,7 +87,7 @@ impl DisplayAs for CometExpandExec { } } -impl ExecutionPlan for CometExpandExec { +impl ExecutionPlan for ExpandExec { fn as_any(&self) -> &dyn Any { self } @@ -104,7 +104,7 @@ impl ExecutionPlan for CometExpandExec { self: Arc, children: Vec>, ) -> datafusion_common::Result> { - let new_expand = CometExpandExec::new( + let new_expand = ExpandExec::new( self.projections.clone(), Arc::clone(&children[0]), Arc::clone(&self.schema), diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index bdc233e942..4e15e4341a 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -27,6 +27,8 @@ pub use filter::FilterExec; pub use scan::*; mod copy; +mod expand; +pub use expand::ExpandExec; mod filter; mod scan; diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 0d35859dfb..a297f87c1f 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -18,8 +18,7 @@ use crate::{ errors::CometError, execution::{ - datafusion::planner::TEST_EXEC_CONTEXT_ID, operators::ExecutionError, - utils::SparkArrowConvert, + operators::ExecutionError, planner::TEST_EXEC_CONTEXT_ID, utils::SparkArrowConvert, }, jvm_bridge::{jni_call, JVMClasses}, }; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/planner.rs similarity index 98% rename from native/core/src/execution/datafusion/planner.rs rename to native/core/src/execution/planner.rs index 0e64ed6afa..3ac830c04e 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/planner.rs @@ -22,17 +22,13 @@ use crate::execution::operators::{CopyMode, FilterExec}; use crate::{ errors::ExpressionError, execution::{ - datafusion::{ - expressions::{ - bloom_filter_agg::BloomFilterAgg, - bloom_filter_might_contain::BloomFilterMightContain, checkoverflow::CheckOverflow, - negative, subquery::Subquery, unbound::UnboundColumn, - }, - operators::expand::CometExpandExec, - shuffle_writer::ShuffleWriterExec, + expressions::{ + bloom_filter_agg::BloomFilterAgg, bloom_filter_might_contain::BloomFilterMightContain, + subquery::Subquery, }, - operators::{CopyExec, ExecutionError, ScanExec}, + operators::{CopyExec, ExecutionError, ExpandExec, ScanExec}, serde::to_arrow_datatype, + shuffle::ShuffleWriterExec, }, }; use arrow::compute::CastOptions; @@ -68,11 +64,11 @@ use datafusion::{ }, prelude::SessionContext, }; -use datafusion_comet_spark_expr::create_comet_physical_fun; +use datafusion_comet_spark_expr::{create_comet_physical_fun, create_negate_expr}; use datafusion_functions_nested::concat::ArrayAppend; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; -use crate::execution::datafusion::spark_plan::SparkPlan; +use crate::execution::spark_plan::SparkPlan; use datafusion_comet_proto::{ spark_expression::{ self, agg_expr::ExprStruct as AggExprStruct, expr::ExprStruct, literal::Value, AggExpr, @@ -86,11 +82,11 @@ use datafusion_comet_proto::{ spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; use datafusion_comet_spark_expr::{ - ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Contains, Correlation, Covariance, - CreateNamedStruct, DateTruncExpr, EndsWith, GetArrayStructFields, GetStructField, HourExpr, - IfExpr, Like, ListExtract, MinuteExpr, NormalizeNaNAndZero, RLike, SecondExpr, + ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, CheckOverflow, Contains, Correlation, + Covariance, CreateNamedStruct, DateTruncExpr, EndsWith, GetArrayStructFields, GetStructField, + HourExpr, IfExpr, Like, ListExtract, MinuteExpr, NormalizeNaNAndZero, RLike, SecondExpr, SparkCastOptions, StartsWith, Stddev, StringSpaceExpr, SubstringExpr, SumDecimal, - TimestampTruncExpr, ToJson, Variance, + TimestampTruncExpr, ToJson, UnboundColumn, Variance, }; use datafusion_common::scalar::ScalarStructBuilder; use datafusion_common::{ @@ -611,7 +607,7 @@ impl PhysicalPlanner { ExprStruct::UnaryMinus(expr) => { let child: Arc = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&input_schema))?; - let result = negative::create_negate_expr(child, expr.fail_on_error); + let result = create_negate_expr(child, expr.fail_on_error); result.map_err(|e| ExecutionError::GeneralError(e.to_string())) } ExprStruct::NormalizeNanAndZero(expr) => { @@ -1118,7 +1114,7 @@ impl PhysicalPlanner { } else { Arc::clone(&child.native_plan) }; - let expand = Arc::new(CometExpandExec::new(projections, input, schema)); + let expand = Arc::new(ExpandExec::new(projections, input, schema)); Ok(( scans, Arc::new(SparkPlan::new(spark_plan.plan_id, expand, vec![child])), @@ -2270,7 +2266,7 @@ mod tests { use datafusion::{physical_plan::common::collect, prelude::SessionContext}; use tokio::sync::mpsc; - use crate::execution::{datafusion::planner::PhysicalPlanner, operators::InputBatch}; + use crate::execution::{operators::InputBatch, planner::PhysicalPlanner}; use crate::execution::operators::ExecutionError; use datafusion_comet_proto::{ diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index b052df29b3..8721ead74b 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -18,3 +18,5 @@ mod list; mod map; pub mod row; +mod shuffle_writer; +pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index 17b180e9d0..ce752e68af 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -20,10 +20,10 @@ use crate::{ errors::CometError, execution::{ - datafusion::shuffle_writer::{write_ipc_compressed, Checksum}, shuffle::{ list::{append_list_element, SparkUnsafeArray}, map::{append_map_elements, get_map_key_value_dt, SparkUnsafeMap}, + shuffle_writer::{write_ipc_compressed, Checksum}, }, utils::bytes_to_i128, }, diff --git a/native/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs similarity index 100% rename from native/core/src/execution/datafusion/shuffle_writer.rs rename to native/core/src/execution/shuffle/shuffle_writer.rs diff --git a/native/core/src/execution/datafusion/spark_plan.rs b/native/core/src/execution/spark_plan.rs similarity index 100% rename from native/core/src/execution/datafusion/spark_plan.rs rename to native/core/src/execution/spark_plan.rs diff --git a/native/core/src/execution/datafusion/util/mod.rs b/native/core/src/execution/util/mod.rs similarity index 100% rename from native/core/src/execution/datafusion/util/mod.rs rename to native/core/src/execution/util/mod.rs diff --git a/native/core/src/execution/datafusion/util/spark_bit_array.rs b/native/core/src/execution/util/spark_bit_array.rs similarity index 100% rename from native/core/src/execution/datafusion/util/spark_bit_array.rs rename to native/core/src/execution/util/spark_bit_array.rs diff --git a/native/core/src/execution/datafusion/util/spark_bloom_filter.rs b/native/core/src/execution/util/spark_bloom_filter.rs similarity index 98% rename from native/core/src/execution/datafusion/util/spark_bloom_filter.rs rename to native/core/src/execution/util/spark_bloom_filter.rs index 35fa23b460..2c3af16916 100644 --- a/native/core/src/execution/datafusion/util/spark_bloom_filter.rs +++ b/native/core/src/execution/util/spark_bloom_filter.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::datafusion::util::spark_bit_array; -use crate::execution::datafusion::util::spark_bit_array::SparkBitArray; +use crate::execution::util::spark_bit_array; +use crate::execution::util::spark_bit_array::SparkBitArray; use arrow_array::{ArrowNativeTypeOp, BooleanArray, Int64Array}; use arrow_buffer::ToByteSlice; use datafusion_comet_spark_expr::spark_hash::spark_compatible_murmur3_hash; diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 65517431d2..d0bc2fd9dd 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -29,6 +29,7 @@ edition = { workspace = true } [dependencies] arrow = { workspace = true } arrow-array = { workspace = true } +arrow-buffer = { workspace = true } arrow-data = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/spark-expr/src/checkoverflow.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/checkoverflow.rs rename to native/spark-expr/src/checkoverflow.rs diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 5dff6e0b8f..8a57480587 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -29,6 +29,8 @@ mod bitwise_not; pub use bitwise_not::{bitwise_not, BitwiseNotExpr}; mod avg_decimal; pub use avg_decimal::AvgDecimal; +mod checkoverflow; +pub use checkoverflow::CheckOverflow; mod correlation; pub use correlation::Correlation; mod covariance; @@ -45,10 +47,14 @@ pub use stddev::Stddev; mod structs; mod sum_decimal; pub use sum_decimal::SumDecimal; +mod negative; +pub use negative::{create_negate_expr, NegativeExpr}; mod normalize_nan; mod temporal; pub mod timezone; mod to_json; +mod unbound; +pub use unbound::UnboundColumn; pub mod utils; pub use normalize_nan::NormalizeNaNAndZero; @@ -83,3 +89,9 @@ pub enum EvalMode { /// failing the entire query. Try, } + +pub(crate) fn arithmetic_overflow_error(from_type: &str) -> SparkError { + SparkError::ArithmeticOverflow { + from_type: from_type.to_string(), + } +} diff --git a/native/core/src/execution/datafusion/expressions/negative.rs b/native/spark-expr/src/negative.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/negative.rs rename to native/spark-expr/src/negative.rs index 8dfe717422..3d9063e783 100644 --- a/native/core/src/execution/datafusion/expressions/negative.rs +++ b/native/spark-expr/src/negative.rs @@ -16,7 +16,7 @@ // under the License. use super::arithmetic_overflow_error; -use crate::errors::CometError; +use crate::SparkError; use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeType}; use arrow_array::RecordBatch; use arrow_buffer::IntervalDayTime; @@ -26,8 +26,7 @@ use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, }; -use datafusion_comet_spark_expr::SparkError; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; use std::{ any::Any, @@ -38,7 +37,7 @@ use std::{ pub fn create_negate_expr( expr: Arc, fail_on_error: bool, -) -> Result, CometError> { +) -> Result, DataFusionError> { Ok(Arc::new(NegativeExpr::new(expr, fail_on_error))) } diff --git a/native/core/src/execution/datafusion/expressions/unbound.rs b/native/spark-expr/src/unbound.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/unbound.rs rename to native/spark-expr/src/unbound.rs From f1d08791e0603e5543702fa952365d4e61f8df4c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 12 Dec 2024 16:41:12 -0700 Subject: [PATCH 34/47] chore: Add ignored tests for reading complex types from Parquet (#1167) * Add ignored tests for reading structs from Parquet * add basic map test * add tests for Map and Array --- .../apache/comet/CometExpressionSuite.scala | 127 ++++++++++++++++++ 1 file changed, 127 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 35f374bf0f..cce7cb20a1 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -2195,6 +2195,133 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + ignore("get_struct_field - select primitive fields") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Add both a null struct and null inner value + .select(when(col("id") > 1, struct(when(col("id") > 2, col("id")).alias("id"))) + .alias("nested1")) + + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("nested1.id")) + } + } + } + } + + ignore("get_struct_field - select subset of struct") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Add both a null struct and null inner value + .select( + when( + col("id") > 1, + struct( + when(col("id") > 2, col("id")).alias("id"), + when(col("id") > 2, struct(when(col("id") > 3, col("id")).alias("id"))) + .as("nested2"))) + .alias("nested1")) + + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("nested1.id")) + checkSparkAnswerAndOperator(df.select("nested1.nested2")) + checkSparkAnswerAndOperator(df.select("nested1.nested2.id")) + checkSparkAnswerAndOperator(df.select("nested1.id", "nested1.nested2.id")) + } + } + } + } + + ignore("get_struct_field - read entire struct") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Add both a null struct and null inner value + .select( + when( + col("id") > 1, + struct( + when(col("id") > 2, col("id")).alias("id"), + when(col("id") > 2, struct(when(col("id") > 3, col("id")).alias("id"))) + .as("nested2"))) + .alias("nested1")) + + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("nested1")) + } + } + } + } + + ignore("read map[int, int] from parquet") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Spark does not allow null as a key but does allow null as a + // value, and the entire map be null + .select( + when(col("id") > 1, map(col("id"), when(col("id") > 2, col("id")))).alias("map1")) + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("map1")) + checkSparkAnswerAndOperator(df.select(map_keys(col("map1")))) + checkSparkAnswerAndOperator(df.select(map_values(col("map1")))) + } + } + } + } + + ignore("read array[int] from parquet") { + withTempPath { dir => + // create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) + // Spark does not allow null as a key but does allow null as a + // value, and the entire map be null + .select(when(col("id") > 1, sequence(lit(0), col("id") * 2)).alias("array1")) + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("array1")) + checkSparkAnswerAndOperator(df.select(element_at(col("array1"), lit(1)))) + } + } + } + } + test("CreateArray") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => From b9ac78beffe8b71388ceb8d7e842fd6a07395829 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 17 Dec 2024 16:35:54 -0700 Subject: [PATCH 35/47] feat: Add Spark-compatible implementation of SchemaAdapterFactory (#1169) * Add Spark-compatible SchemaAdapterFactory implementation * remove prototype code * fix * refactor * implement more cast logic * implement more cast logic * add basic test * improve test * cleanup * fmt * add support for casting unsigned int to signed int * clippy * address feedback * fix test --- native/Cargo.lock | 67 +++- native/Cargo.toml | 2 +- .../core/src/parquet/util/test_common/mod.rs | 3 +- native/spark-expr/Cargo.toml | 4 +- native/spark-expr/src/cast.rs | 353 +++++++++++++--- native/spark-expr/src/lib.rs | 5 + native/spark-expr/src/schema_adapter.rs | 376 ++++++++++++++++++ .../src}/test_common/file_util.rs | 0 native/spark-expr/src/test_common/mod.rs | 17 + 9 files changed, 758 insertions(+), 69 deletions(-) create mode 100644 native/spark-expr/src/schema_adapter.rs rename native/{core/src/parquet/util => spark-expr/src}/test_common/file_util.rs (100%) create mode 100644 native/spark-expr/src/test_common/mod.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index 7966bb80bb..538c40ee23 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -436,7 +436,18 @@ checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", - "brotli-decompressor", + "brotli-decompressor 2.5.1", +] + +[[package]] +name = "brotli" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor 4.0.1", ] [[package]] @@ -449,6 +460,16 @@ dependencies = [ "alloc-stdlib", ] +[[package]] +name = "brotli-decompressor" +version = "4.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a45bd2e4095a8b518033b128020dd4a55aab1c0a381ba4404a472630f4bc362" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + [[package]] name = "bumpalo" version = "3.16.0" @@ -842,6 +863,7 @@ dependencies = [ "num_cpus", "object_store", "parking_lot", + "parquet", "paste", "pin-project-lite", "rand", @@ -878,7 +900,7 @@ dependencies = [ "arrow-schema", "assertables", "async-trait", - "brotli", + "brotli 3.5.0", "bytes", "crc32fast", "criterion", @@ -914,7 +936,7 @@ dependencies = [ "tempfile", "thiserror", "tokio", - "zstd", + "zstd 0.11.2+zstd.1.5.2", ] [[package]] @@ -943,6 +965,7 @@ dependencies = [ "datafusion-physical-expr", "futures", "num", + "parquet", "rand", "regex", "thiserror", @@ -969,6 +992,7 @@ dependencies = [ "libc", "num_cpus", "object_store", + "parquet", "paste", "sqlparser", "tokio", @@ -2350,16 +2374,33 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dea02606ba6f5e856561d8d507dba8bac060aefca2a6c0f1aa1d361fed91ff3e" dependencies = [ "ahash", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-schema", + "arrow-select", + "base64", + "brotli 7.0.0", "bytes", "chrono", + "flate2", + "futures", "half", "hashbrown 0.14.5", + "lz4_flex", "num", "num-bigint", + "object_store", "paste", "seq-macro", + "snap", "thrift", + "tokio", "twox-hash 1.6.3", + "zstd 0.13.2", + "zstd-sys", ] [[package]] @@ -3652,7 +3693,16 @@ version = "0.11.2+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" dependencies = [ - "zstd-safe", + "zstd-safe 5.0.2+zstd.1.5.2", +] + +[[package]] +name = "zstd" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" +dependencies = [ + "zstd-safe 7.2.1", ] [[package]] @@ -3665,6 +3715,15 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "zstd-safe" +version = "7.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" +dependencies = [ + "zstd-sys", +] + [[package]] name = "zstd-sys" version = "2.0.13+zstd.1.5.6" diff --git a/native/Cargo.toml b/native/Cargo.toml index 4ac85479f2..bd46cf0c9f 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,8 +39,8 @@ arrow-buffer = { version = "53.2.0" } arrow-data = { version = "53.2.0" } arrow-schema = { version = "53.2.0" } parquet = { version = "53.2.0", default-features = false, features = ["experimental"] } -datafusion-common = { version = "43.0.0" } datafusion = { version = "43.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { version = "43.0.0" } datafusion-functions = { version = "43.0.0", features = ["crypto_expressions"] } datafusion-functions-nested = { version = "43.0.0", default-features = false } datafusion-expr = { version = "43.0.0", default-features = false } diff --git a/native/core/src/parquet/util/test_common/mod.rs b/native/core/src/parquet/util/test_common/mod.rs index e46d732239..d92544608e 100644 --- a/native/core/src/parquet/util/test_common/mod.rs +++ b/native/core/src/parquet/util/test_common/mod.rs @@ -15,10 +15,9 @@ // specific language governing permissions and limitations // under the License. -pub mod file_util; pub mod page_util; pub mod rand_gen; pub use self::rand_gen::{random_bools, random_bytes, random_numbers, random_numbers_range}; -pub use self::file_util::{get_temp_file, get_temp_filename}; +pub use datafusion_comet_spark_expr::test_common::file_util::{get_temp_file, get_temp_filename}; diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index d0bc2fd9dd..27367d83e1 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -33,7 +33,7 @@ arrow-buffer = { workspace = true } arrow-data = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } -datafusion = { workspace = true } +datafusion = { workspace = true, features = ["parquet"] } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-expr = { workspace = true } @@ -43,9 +43,11 @@ regex = { workspace = true } thiserror = { workspace = true } futures = { workspace = true } twox-hash = "2.0.0" +rand = { workspace = true } [dev-dependencies] arrow-data = {workspace = true} +parquet = { workspace = true, features = ["arrow"] } criterion = "0.5.1" rand = { workspace = true} tokio = { version = "1", features = ["rt-multi-thread"] } diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index f62d0220c9..d96bcbbdb6 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use crate::timezone; +use crate::utils::array_with_timezone; +use crate::{EvalMode, SparkError, SparkResult}; use arrow::{ array::{ cast::AsArray, @@ -35,11 +38,18 @@ use arrow::{ use arrow_array::builder::StringBuilder; use arrow_array::{DictionaryArray, StringArray, StructArray}; use arrow_schema::{DataType, Field, Schema}; +use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; +use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, }; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; +use num::{ + cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, + ToPrimitive, +}; +use regex::Regex; use std::str::FromStr; use std::{ any::Any, @@ -49,19 +59,6 @@ use std::{ sync::Arc, }; -use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; -use num::{ - cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, - ToPrimitive, -}; -use regex::Regex; - -use crate::timezone; -use crate::utils::array_with_timezone; - -use crate::{EvalMode, SparkError, SparkResult}; - static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f"); const MICROS_PER_SECOND: i64 = 1000000; @@ -141,6 +138,240 @@ pub struct Cast { pub cast_options: SparkCastOptions, } +/// Determine if Comet supports a cast, taking options such as EvalMode and Timezone into account. +pub fn cast_supported( + from_type: &DataType, + to_type: &DataType, + options: &SparkCastOptions, +) -> bool { + use DataType::*; + + let from_type = if let Dictionary(_, dt) = from_type { + dt + } else { + from_type + }; + + let to_type = if let Dictionary(_, dt) = to_type { + dt + } else { + to_type + }; + + if from_type == to_type { + return true; + } + + match (from_type, to_type) { + (Boolean, _) => can_cast_from_boolean(to_type, options), + (UInt8 | UInt16 | UInt32 | UInt64, Int8 | Int16 | Int32 | Int64) + if options.allow_cast_unsigned_ints => + { + true + } + (Int8, _) => can_cast_from_byte(to_type, options), + (Int16, _) => can_cast_from_short(to_type, options), + (Int32, _) => can_cast_from_int(to_type, options), + (Int64, _) => can_cast_from_long(to_type, options), + (Float32, _) => can_cast_from_float(to_type, options), + (Float64, _) => can_cast_from_double(to_type, options), + (Decimal128(p, s), _) => can_cast_from_decimal(p, s, to_type, options), + (Timestamp(_, None), _) => can_cast_from_timestamp_ntz(to_type, options), + (Timestamp(_, Some(_)), _) => can_cast_from_timestamp(to_type, options), + (Utf8 | LargeUtf8, _) => can_cast_from_string(to_type, options), + (_, Utf8 | LargeUtf8) => can_cast_to_string(from_type, options), + (Struct(from_fields), Struct(to_fields)) => from_fields + .iter() + .zip(to_fields.iter()) + .all(|(a, b)| cast_supported(a.data_type(), b.data_type(), options)), + _ => false, + } +} + +fn can_cast_from_string(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Binary => true, + Float32 | Float64 => { + // https://github.com/apache/datafusion-comet/issues/326 + // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'. + // Does not support ANSI mode. + options.allow_incompat + } + Decimal128(_, _) => { + // https://github.com/apache/datafusion-comet/issues/325 + // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'. + // Does not support ANSI mode. Returns 0.0 instead of null if input contains no digits + + options.allow_incompat + } + Date32 | Date64 => { + // https://github.com/apache/datafusion-comet/issues/327 + // Only supports years between 262143 BC and 262142 AD + options.allow_incompat + } + Timestamp(_, _) if options.eval_mode == EvalMode::Ansi => { + // ANSI mode not supported + false + } + Timestamp(_, Some(tz)) if tz.as_ref() != "UTC" => { + // Cast will use UTC instead of $timeZoneId + options.allow_incompat + } + Timestamp(_, _) => { + // https://github.com/apache/datafusion-comet/issues/328 + // Not all valid formats are supported + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_to_string(from_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match from_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Date32 | Date64 | Timestamp(_, _) => true, + Float32 | Float64 => { + // There can be differences in precision. + // For example, the input \"1.4E-45\" will produce 1.0E-45 " + + // instead of 1.4E-45")) + true + } + Decimal128(_, _) => { + // https://github.com/apache/datafusion-comet/issues/1068 + // There can be formatting differences in some case due to Spark using + // scientific notation where Comet does not + true + } + Binary => { + // https://github.com/apache/datafusion-comet/issues/377 + // Only works for binary data representing valid UTF-8 strings + options.allow_incompat + } + Struct(fields) => fields + .iter() + .all(|f| can_cast_to_string(f.data_type(), options)), + _ => false, + } +} + +fn can_cast_from_timestamp_ntz(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Timestamp(_, _) | Date32 | Date64 | Utf8 => { + // incompatible + options.allow_incompat + } + _ => { + // unsupported + false + } + } +} + +fn can_cast_from_timestamp(to_type: &DataType, _options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 => { + // https://github.com/apache/datafusion-comet/issues/352 + // this seems like an edge case that isn't important for us to support + false + } + Int64 => { + // https://github.com/apache/datafusion-comet/issues/352 + true + } + Date32 | Date64 | Utf8 | Decimal128(_, _) => true, + _ => { + // unsupported + false + } + } +} + +fn can_cast_from_boolean(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!(to_type, Int8 | Int16 | Int32 | Int64 | Float32 | Float64) +} + +fn can_cast_from_byte(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_short(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_int(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Utf8 => true, + Decimal128(_, _) => { + // incompatible: no overflow check + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_from_long(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true, + Decimal128(_, _) => { + // incompatible: no overflow check + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_from_float(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_double(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Decimal128(_, _) + ) +} + +fn can_cast_from_decimal( + p1: &u8, + _s1: &i8, + to_type: &DataType, + options: &SparkCastOptions, +) -> bool { + use DataType::*; + match to_type { + Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true, + Decimal128(p2, _) => { + if p2 < p1 { + // https://github.com/apache/datafusion/issues/13492 + // Incompatible(Some("Casting to smaller precision is not supported")) + options.allow_incompat + } else { + true + } + } + _ => false, + } +} + macro_rules! cast_utf8_to_int { ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident) => {{ let len = $array.len(); @@ -560,6 +791,8 @@ pub struct SparkCastOptions { pub timezone: String, /// Allow casts that are supported but not guaranteed to be 100% compatible pub allow_incompat: bool, + /// Support casting unsigned ints to signed ints (used by Parquet SchemaAdapter) + pub allow_cast_unsigned_ints: bool, } impl SparkCastOptions { @@ -568,6 +801,7 @@ impl SparkCastOptions { eval_mode, timezone: timezone.to_string(), allow_incompat, + allow_cast_unsigned_ints: false, } } @@ -576,6 +810,7 @@ impl SparkCastOptions { eval_mode, timezone: "".to_string(), allow_incompat, + allow_cast_unsigned_ints: false, } } } @@ -611,14 +846,14 @@ fn cast_array( to_type: &DataType, cast_options: &SparkCastOptions, ) -> DataFusionResult { + use DataType::*; let array = array_with_timezone(array, cast_options.timezone.clone(), Some(to_type))?; let from_type = array.data_type().clone(); let array = match &from_type { - DataType::Dictionary(key_type, value_type) - if key_type.as_ref() == &DataType::Int32 - && (value_type.as_ref() == &DataType::Utf8 - || value_type.as_ref() == &DataType::LargeUtf8) => + Dictionary(key_type, value_type) + if key_type.as_ref() == &Int32 + && (value_type.as_ref() == &Utf8 || value_type.as_ref() == &LargeUtf8) => { let dict_array = array .as_any() @@ -631,7 +866,7 @@ fn cast_array( ); let casted_result = match to_type { - DataType::Dictionary(_, _) => Arc::new(casted_dictionary.clone()), + Dictionary(_, _) => Arc::new(casted_dictionary.clone()), _ => take(casted_dictionary.values().as_ref(), dict_array.keys(), None)?, }; return Ok(spark_cast_postprocess(casted_result, &from_type, to_type)); @@ -642,70 +877,66 @@ fn cast_array( let eval_mode = cast_options.eval_mode; let cast_result = match (from_type, to_type) { - (DataType::Utf8, DataType::Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode), - (DataType::LargeUtf8, DataType::Boolean) => { - spark_cast_utf8_to_boolean::(&array, eval_mode) - } - (DataType::Utf8, DataType::Timestamp(_, _)) => { + (Utf8, Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode), + (LargeUtf8, Boolean) => spark_cast_utf8_to_boolean::(&array, eval_mode), + (Utf8, Timestamp(_, _)) => { cast_string_to_timestamp(&array, to_type, eval_mode, &cast_options.timezone) } - (DataType::Utf8, DataType::Date32) => cast_string_to_date(&array, to_type, eval_mode), - (DataType::Int64, DataType::Int32) - | (DataType::Int64, DataType::Int16) - | (DataType::Int64, DataType::Int8) - | (DataType::Int32, DataType::Int16) - | (DataType::Int32, DataType::Int8) - | (DataType::Int16, DataType::Int8) + (Utf8, Date32) => cast_string_to_date(&array, to_type, eval_mode), + (Int64, Int32) + | (Int64, Int16) + | (Int64, Int8) + | (Int32, Int16) + | (Int32, Int8) + | (Int16, Int8) if eval_mode != EvalMode::Try => { spark_cast_int_to_int(&array, eval_mode, from_type, to_type) } - (DataType::Utf8, DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64) => { + (Utf8, Int8 | Int16 | Int32 | Int64) => { cast_string_to_int::(to_type, &array, eval_mode) } - ( - DataType::LargeUtf8, - DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) => cast_string_to_int::(to_type, &array, eval_mode), - (DataType::Float64, DataType::Utf8) => spark_cast_float64_to_utf8::(&array, eval_mode), - (DataType::Float64, DataType::LargeUtf8) => { - spark_cast_float64_to_utf8::(&array, eval_mode) - } - (DataType::Float32, DataType::Utf8) => spark_cast_float32_to_utf8::(&array, eval_mode), - (DataType::Float32, DataType::LargeUtf8) => { - spark_cast_float32_to_utf8::(&array, eval_mode) - } - (DataType::Float32, DataType::Decimal128(precision, scale)) => { + (LargeUtf8, Int8 | Int16 | Int32 | Int64) => { + cast_string_to_int::(to_type, &array, eval_mode) + } + (Float64, Utf8) => spark_cast_float64_to_utf8::(&array, eval_mode), + (Float64, LargeUtf8) => spark_cast_float64_to_utf8::(&array, eval_mode), + (Float32, Utf8) => spark_cast_float32_to_utf8::(&array, eval_mode), + (Float32, LargeUtf8) => spark_cast_float32_to_utf8::(&array, eval_mode), + (Float32, Decimal128(precision, scale)) => { cast_float32_to_decimal128(&array, *precision, *scale, eval_mode) } - (DataType::Float64, DataType::Decimal128(precision, scale)) => { + (Float64, Decimal128(precision, scale)) => { cast_float64_to_decimal128(&array, *precision, *scale, eval_mode) } - (DataType::Float32, DataType::Int8) - | (DataType::Float32, DataType::Int16) - | (DataType::Float32, DataType::Int32) - | (DataType::Float32, DataType::Int64) - | (DataType::Float64, DataType::Int8) - | (DataType::Float64, DataType::Int16) - | (DataType::Float64, DataType::Int32) - | (DataType::Float64, DataType::Int64) - | (DataType::Decimal128(_, _), DataType::Int8) - | (DataType::Decimal128(_, _), DataType::Int16) - | (DataType::Decimal128(_, _), DataType::Int32) - | (DataType::Decimal128(_, _), DataType::Int64) + (Float32, Int8) + | (Float32, Int16) + | (Float32, Int32) + | (Float32, Int64) + | (Float64, Int8) + | (Float64, Int16) + | (Float64, Int32) + | (Float64, Int64) + | (Decimal128(_, _), Int8) + | (Decimal128(_, _), Int16) + | (Decimal128(_, _), Int32) + | (Decimal128(_, _), Int64) if eval_mode != EvalMode::Try => { spark_cast_nonintegral_numeric_to_integral(&array, eval_mode, from_type, to_type) } - (DataType::Struct(_), DataType::Utf8) => { - Ok(casts_struct_to_string(array.as_struct(), cast_options)?) - } - (DataType::Struct(_), DataType::Struct(_)) => Ok(cast_struct_to_struct( + (Struct(_), Utf8) => Ok(casts_struct_to_string(array.as_struct(), cast_options)?), + (Struct(_), Struct(_)) => Ok(cast_struct_to_struct( array.as_struct(), from_type, to_type, cast_options, )?), + (UInt8 | UInt16 | UInt32 | UInt64, Int8 | Int16 | Int32 | Int64) + if cast_options.allow_cast_unsigned_ints => + { + Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) + } _ if is_datafusion_spark_compatible(from_type, to_type, cast_options.allow_incompat) => { // use DataFusion cast only when we know that it is compatible with Spark Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 8a57480587..f358731004 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -41,6 +41,9 @@ mod kernels; mod list; mod regexp; pub mod scalar_funcs; +mod schema_adapter; +pub use schema_adapter::SparkSchemaAdapterFactory; + pub mod spark_hash; mod stddev; pub use stddev::Stddev; @@ -51,6 +54,8 @@ mod negative; pub use negative::{create_negate_expr, NegativeExpr}; mod normalize_nan; mod temporal; + +pub mod test_common; pub mod timezone; mod to_json; mod unbound; diff --git a/native/spark-expr/src/schema_adapter.rs b/native/spark-expr/src/schema_adapter.rs new file mode 100644 index 0000000000..161ad6f164 --- /dev/null +++ b/native/spark-expr/src/schema_adapter.rs @@ -0,0 +1,376 @@ +// 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. + +//! Custom schema adapter that uses Spark-compatible casts + +use crate::cast::cast_supported; +use crate::{spark_cast, SparkCastOptions}; +use arrow_array::{new_null_array, Array, RecordBatch, RecordBatchOptions}; +use arrow_schema::{Schema, SchemaRef}; +use datafusion::datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory, SchemaMapper}; +use datafusion_common::plan_err; +use datafusion_expr::ColumnarValue; +use std::sync::Arc; + +/// An implementation of DataFusion's `SchemaAdapterFactory` that uses a Spark-compatible +/// `cast` implementation. +#[derive(Clone, Debug)] +pub struct SparkSchemaAdapterFactory { + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SparkSchemaAdapterFactory { + pub fn new(options: SparkCastOptions) -> Self { + Self { + cast_options: options, + } + } +} + +impl SchemaAdapterFactory for SparkSchemaAdapterFactory { + /// Create a new factory for mapping batches from a file schema to a table + /// schema. + /// + /// This is a convenience for [`DefaultSchemaAdapterFactory::create`] with + /// the same schema for both the projected table schema and the table + /// schema. + fn create( + &self, + required_schema: SchemaRef, + table_schema: SchemaRef, + ) -> Box { + Box::new(SparkSchemaAdapter { + required_schema, + table_schema, + cast_options: self.cast_options.clone(), + }) + } +} + +/// This SchemaAdapter requires both the table schema and the projected table +/// schema. See [`SchemaMapping`] for more details +#[derive(Clone, Debug)] +pub struct SparkSchemaAdapter { + /// The schema for the table, projected to include only the fields being output (projected) by the + /// associated ParquetExec + required_schema: SchemaRef, + /// The entire table schema for the table we're using this to adapt. + /// + /// This is used to evaluate any filters pushed down into the scan + /// which may refer to columns that are not referred to anywhere + /// else in the plan. + table_schema: SchemaRef, + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SchemaAdapter for SparkSchemaAdapter { + /// Map a column index in the table schema to a column index in a particular + /// file schema + /// + /// Panics if index is not in range for the table schema + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.required_schema.field(index); + Some(file_schema.fields.find(field.name())?.0) + } + + /// Creates a `SchemaMapping` for casting or mapping the columns from the + /// file schema to the table schema. + /// + /// If the provided `file_schema` contains columns of a different type to + /// the expected `table_schema`, the method will attempt to cast the array + /// data from the file schema to the table schema where possible. + /// + /// Returns a [`SchemaMapping`] that can be applied to the output batch + /// along with an ordered list of columns to project from the file + fn map_schema( + &self, + file_schema: &Schema, + ) -> datafusion_common::Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + let mut field_mappings = vec![None; self.required_schema.fields().len()]; + + for (file_idx, file_field) in file_schema.fields.iter().enumerate() { + if let Some((table_idx, table_field)) = + self.required_schema.fields().find(file_field.name()) + { + if cast_supported( + file_field.data_type(), + table_field.data_type(), + &self.cast_options, + ) { + field_mappings[table_idx] = Some(projection.len()); + projection.push(file_idx); + } else { + return plan_err!( + "Cannot cast file schema field {} of type {:?} to required schema field of type {:?}", + file_field.name(), + file_field.data_type(), + table_field.data_type() + ); + } + } + } + + Ok(( + Arc::new(SchemaMapping { + required_schema: Arc::::clone(&self.required_schema), + field_mappings, + table_schema: Arc::::clone(&self.table_schema), + cast_options: self.cast_options.clone(), + }), + projection, + )) + } +} + +// TODO SchemaMapping is mostly copied from DataFusion but calls spark_cast +// instead of arrow cast - can we reduce the amount of code copied here and make +// the DataFusion version more extensible? + +/// The SchemaMapping struct holds a mapping from the file schema to the table +/// schema and any necessary type conversions. +/// +/// Note, because `map_batch` and `map_partial_batch` functions have different +/// needs, this struct holds two schemas: +/// +/// 1. The projected **table** schema +/// 2. The full table schema +/// +/// [`map_batch`] is used by the ParquetOpener to produce a RecordBatch which +/// has the projected schema, since that's the schema which is supposed to come +/// out of the execution of this query. Thus `map_batch` uses +/// `projected_table_schema` as it can only operate on the projected fields. +/// +/// [`map_partial_batch`] is used to create a RecordBatch with a schema that +/// can be used for Parquet predicate pushdown, meaning that it may contain +/// fields which are not in the projected schema (as the fields that parquet +/// pushdown filters operate can be completely distinct from the fields that are +/// projected (output) out of the ParquetExec). `map_partial_batch` thus uses +/// `table_schema` to create the resulting RecordBatch (as it could be operating +/// on any fields in the schema). +/// +/// [`map_batch`]: Self::map_batch +/// [`map_partial_batch`]: Self::map_partial_batch +#[derive(Debug)] +pub struct SchemaMapping { + /// The schema of the table. This is the expected schema after conversion + /// and it should match the schema of the query result. + required_schema: SchemaRef, + /// Mapping from field index in `projected_table_schema` to index in + /// projected file_schema. + /// + /// They are Options instead of just plain `usize`s because the table could + /// have fields that don't exist in the file. + field_mappings: Vec>, + /// The entire table schema, as opposed to the projected_table_schema (which + /// only contains the columns that we are projecting out of this query). + /// This contains all fields in the table, regardless of if they will be + /// projected out or not. + table_schema: SchemaRef, + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SchemaMapper for SchemaMapping { + /// Adapts a `RecordBatch` to match the `projected_table_schema` using the stored mapping and + /// conversions. The produced RecordBatch has a schema that contains only the projected + /// columns, so if one needs a RecordBatch with a schema that references columns which are not + /// in the projected, it would be better to use `map_partial_batch` + fn map_batch(&self, batch: RecordBatch) -> datafusion_common::Result { + let batch_rows = batch.num_rows(); + let batch_cols = batch.columns().to_vec(); + + let cols = self + .required_schema + // go through each field in the projected schema + .fields() + .iter() + // and zip it with the index that maps fields from the projected table schema to the + // projected file schema in `batch` + .zip(&self.field_mappings) + // and for each one... + .map(|(field, file_idx)| { + file_idx.map_or_else( + // If this field only exists in the table, and not in the file, then we know + // that it's null, so just return that. + || Ok(new_null_array(field.data_type(), batch_rows)), + // However, if it does exist in both, then try to cast it to the correct output + // type + |batch_idx| { + spark_cast( + ColumnarValue::Array(Arc::clone(&batch_cols[batch_idx])), + field.data_type(), + &self.cast_options, + )? + .into_array(batch_rows) + }, + ) + }) + .collect::, _>>()?; + + // Necessary to handle empty batches + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + + let schema = Arc::::clone(&self.required_schema); + let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; + Ok(record_batch) + } + + /// Adapts a [`RecordBatch`]'s schema into one that has all the correct output types and only + /// contains the fields that exist in both the file schema and table schema. + /// + /// Unlike `map_batch` this method also preserves the columns that + /// may not appear in the final output (`projected_table_schema`) but may + /// appear in push down predicates + fn map_partial_batch(&self, batch: RecordBatch) -> datafusion_common::Result { + let batch_cols = batch.columns().to_vec(); + let schema = batch.schema(); + + // for each field in the batch's schema (which is based on a file, not a table)... + let (cols, fields) = schema + .fields() + .iter() + .zip(batch_cols.iter()) + .flat_map(|(field, batch_col)| { + self.table_schema + // try to get the same field from the table schema that we have stored in self + .field_with_name(field.name()) + // and if we don't have it, that's fine, ignore it. This may occur when we've + // created an external table whose fields are a subset of the fields in this + // file, then tried to read data from the file into this table. If that is the + // case here, it's fine to ignore because we don't care about this field + // anyways + .ok() + // but if we do have it, + .map(|table_field| { + // try to cast it into the correct output type. we don't want to ignore this + // error, though, so it's propagated. + spark_cast( + ColumnarValue::Array(Arc::clone(batch_col)), + table_field.data_type(), + &self.cast_options, + )? + .into_array(batch_col.len()) + // and if that works, return the field and column. + .map(|new_col| (new_col, table_field.clone())) + }) + }) + .collect::, _>>()? + .into_iter() + .unzip::<_, _, Vec<_>, Vec<_>>(); + + // Necessary to handle empty batches + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + + let schema = Arc::new(Schema::new_with_metadata(fields, schema.metadata().clone())); + let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; + Ok(record_batch) + } +} + +#[cfg(test)] +mod test { + use crate::test_common::file_util::get_temp_filename; + use crate::{EvalMode, SparkCastOptions, SparkSchemaAdapterFactory}; + use arrow::array::{Int32Array, StringArray}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use arrow_array::UInt32Array; + use arrow_schema::SchemaRef; + use datafusion::datasource::listing::PartitionedFile; + use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; + use datafusion::execution::object_store::ObjectStoreUrl; + use datafusion::execution::TaskContext; + use datafusion::physical_plan::ExecutionPlan; + use datafusion_common::DataFusionError; + use futures::StreamExt; + use parquet::arrow::ArrowWriter; + use std::fs::File; + use std::sync::Arc; + + #[tokio::test] + async fn parquet_roundtrip_int_as_string() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let ids = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let names = Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])) + as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids, names])?; + + let required_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("name", DataType::Utf8, false), + ])); + + let _ = roundtrip(&batch, required_schema).await?; + + Ok(()) + } + + #[tokio::test] + async fn parquet_roundtrip_unsigned_int() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::UInt32, false)])); + + let ids = Arc::new(UInt32Array::from(vec![1, 2, 3])) as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids])?; + + let required_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let _ = roundtrip(&batch, required_schema).await?; + + Ok(()) + } + + /// Create a Parquet file containing a single batch and then read the batch back using + /// the specified required_schema. This will cause the SchemaAdapter code to be used. + async fn roundtrip( + batch: &RecordBatch, + required_schema: SchemaRef, + ) -> Result { + let filename = get_temp_filename(); + let filename = filename.as_path().as_os_str().to_str().unwrap().to_string(); + let file = File::create(&filename)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&batch.schema()), None)?; + writer.write(batch)?; + writer.close()?; + + let object_store_url = ObjectStoreUrl::local_filesystem(); + let file_scan_config = FileScanConfig::new(object_store_url, required_schema) + .with_file_groups(vec![vec![PartitionedFile::from_path( + filename.to_string(), + )?]]); + + let mut spark_cast_options = SparkCastOptions::new(EvalMode::Legacy, "UTC", false); + spark_cast_options.allow_cast_unsigned_ints = true; + + let parquet_exec = ParquetExec::builder(file_scan_config) + .with_schema_adapter_factory(Arc::new(SparkSchemaAdapterFactory::new( + spark_cast_options, + ))) + .build(); + + let mut stream = parquet_exec + .execute(0, Arc::new(TaskContext::default())) + .unwrap(); + stream.next().await.unwrap() + } +} diff --git a/native/core/src/parquet/util/test_common/file_util.rs b/native/spark-expr/src/test_common/file_util.rs similarity index 100% rename from native/core/src/parquet/util/test_common/file_util.rs rename to native/spark-expr/src/test_common/file_util.rs diff --git a/native/spark-expr/src/test_common/mod.rs b/native/spark-expr/src/test_common/mod.rs new file mode 100644 index 0000000000..efd25a4a2a --- /dev/null +++ b/native/spark-expr/src/test_common/mod.rs @@ -0,0 +1,17 @@ +// 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. +pub mod file_util; From 46a28db5e5014fc7fabe8eafc7dbeb5027974c47 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 17 Dec 2024 15:50:36 -0800 Subject: [PATCH 36/47] fix: Document enabling comet explain plan usage in Spark (4.0) (#1176) --- docs/source/user-guide/tuning.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index af722494f9..f10a0dde9e 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -127,3 +127,13 @@ Here is a guide to some of the native metrics. | `elapsed_compute` | Total time spent in this operator, fetching batches from a JVM iterator. | | `jvm_fetch_time` | Time spent in the JVM fetching input batches to be read by this `ScanExec` instance. | | `arrow_ffi_time` | Time spent using Arrow FFI to create Arrow batches from the memory addresses returned from the JVM. | + +## Explain Plan +### Extended Explain +With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists +reasons why Comet may not have been enabled for specific operations. +To enable this, in the Spark configuration, set the following: +```shell +-c spark.sql.extendedExplainProviders=org.apache.comet.ExtendedExplainInfo +``` +This will add a section to the detailed plan displayed in the Spark SQL UI page. \ No newline at end of file From 655081b66808d8f70abebd4d85af7c401c579a3a Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA Date: Wed, 18 Dec 2024 09:10:34 -0800 Subject: [PATCH 37/47] test: enabling Spark tests with offHeap requirement (#1177) ## Which issue does this PR close? ## Rationale for this change After https://github.com/apache/datafusion-comet/pull/1062 We have not running Spark tests for native execution ## What changes are included in this PR? Removed the off heap requirement for testing ## How are these changes tested? Bringing back Spark tests for native execution --- dev/diffs/4.0.0-preview1.diff | 103 +++++++++++++++++- native/core/src/execution/jni_api.rs | 26 ++++- .../org/apache/comet/CometExecIterator.scala | 9 +- .../comet/CometSparkSessionExtensions.scala | 10 +- .../main/scala/org/apache/comet/Native.scala | 5 + 5 files changed, 142 insertions(+), 11 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index ba68d2a7bd..db62ed607e 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -146,6 +146,77 @@ index 698ca009b4f..57d774a3617 100644 -- Test tables CREATE table explain_temp1 (key int, val int) USING PARQUET; +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +index 3a409eea348..26e9aaf215c 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +@@ -6,6 +6,9 @@ + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int4.sql + -- + ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + CREATE TABLE INT4_TBL(f1 int) USING parquet; + + -- [SPARK-28023] Trim the string when cast string type to other types +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +index fac23b4a26f..98b12ae5ccc 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +@@ -6,6 +6,10 @@ + -- Test int8 64-bit integers. + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int8.sql + -- ++ ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet; + + -- PostgreSQL implicitly casts string literals to data with integral types, but +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql +index 0efe0877e9b..f9df0400c99 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql +@@ -6,6 +6,9 @@ + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql + -- + ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + -- load test data + CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; + INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A'); +diff --git a/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql b/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql +index e803254ea64..74db78aee38 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/view-schema-binding-config.sql +@@ -1,6 +1,9 @@ + -- This test suits check the spark.sql.viewSchemaBindingMode configuration. + -- It can be DISABLED and COMPENSATION + ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + -- Verify the default binding is true + SET spark.sql.legacy.viewSchemaBindingMode; + +diff --git a/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql b/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql +index 21a3ce1e122..f4762ab98f0 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/view-schema-compensation.sql +@@ -1,5 +1,9 @@ + -- This test suite checks the WITH SCHEMA COMPENSATION clause + -- Disable ANSI mode to ensure we are forcing it explicitly in the CASTS ++ ++-- TODO: https://github.com/apache/datafusion-comet/issues/551 ++--SET spark.comet.enabled = false ++ + SET spark.sql.ansi.enabled = false; + + -- In COMPENSATION views get invalidated if the type can't cast diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index d023fb82185..0f4f03bda6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -917,7 +988,7 @@ index 34c6c49bc49..f5dea07a213 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 56c364e2084..a00a50e020a 100644 +index 56c364e2084..fc3abd7cdc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1510,7 +1510,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -930,6 +1001,36 @@ index 56c364e2084..a00a50e020a 100644 AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } +@@ -4454,7 +4455,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + } + + test("SPARK-39166: Query context of binary arithmetic should be serialized to executors" + +- " when WSCG is off") { ++ " when WSCG is off", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.ANSI_ENABLED.key -> "true") { + withTable("t") { +@@ -4475,7 +4477,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + } + + test("SPARK-39175: Query context of Cast should be serialized to executors" + +- " when WSCG is off") { ++ " when WSCG is off", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.ANSI_ENABLED.key -> "true") { + withTable("t") { +@@ -4502,7 +4505,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + } + + test("SPARK-39190,SPARK-39208,SPARK-39210: Query context of decimal overflow error should " + +- "be serialized to executors when WSCG is off") { ++ "be serialized to executors when WSCG is off", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.ANSI_ENABLED.key -> "true") { + withTable("t") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 68f14f13bbd..174636cefb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 491b389c99..eb73675b5b 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -106,6 +106,9 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( metrics_node: JObject, comet_task_memory_manager_obj: JObject, batch_size: jint, + use_unified_memory_manager: jboolean, + memory_limit: jlong, + memory_fraction: jdouble, debug_native: jboolean, explain_native: jboolean, worker_threads: jint, @@ -147,7 +150,13 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( // We need to keep the session context alive. Some session state like temporary // dictionaries are stored in session context. If it is dropped, the temporary // dictionaries will be dropped as well. - let session = prepare_datafusion_session_context(batch_size as usize, task_memory_manager)?; + let session = prepare_datafusion_session_context( + batch_size as usize, + use_unified_memory_manager == 1, + memory_limit as usize, + memory_fraction, + task_memory_manager, + )?; let plan_creation_time = start.elapsed(); @@ -174,13 +183,22 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( /// Configure DataFusion session context. fn prepare_datafusion_session_context( batch_size: usize, + use_unified_memory_manager: bool, + memory_limit: usize, + memory_fraction: f64, comet_task_memory_manager: Arc, ) -> CometResult { let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); - // Set Comet memory pool for native - let memory_pool = CometMemoryPool::new(comet_task_memory_manager); - rt_config = rt_config.with_memory_pool(Arc::new(memory_pool)); + // Check if we are using unified memory manager integrated with Spark. + if use_unified_memory_manager { + // Set Comet memory pool for native + let memory_pool = CometMemoryPool::new(comet_task_memory_manager); + rt_config = rt_config.with_memory_pool(Arc::new(memory_pool)); + } else { + // Use the memory pool from DF + rt_config = rt_config.with_memory_limit(memory_limit, memory_fraction) + } // Get Datafusion configuration from Spark Execution context // can be configured in Comet Spark JVM using Spark --conf parameters diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index d57e9e2b87..04d9306951 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -23,7 +23,7 @@ import org.apache.spark._ import org.apache.spark.sql.comet.CometMetricNode import org.apache.spark.sql.vectorized._ -import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} +import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} import org.apache.comet.vector.NativeUtil /** @@ -60,6 +60,10 @@ class CometExecIterator( new CometBatchIterator(iterator, nativeUtil) }.toArray private val plan = { + val conf = SparkEnv.get.conf + // Only enable unified memory manager when off-heap mode is enabled. Otherwise, + // we'll use the built-in memory pool from DF, and initializes with `memory_limit` + // and `memory_fraction` below. nativeLib.createPlan( id, cometBatchIterators, @@ -67,6 +71,9 @@ class CometExecIterator( nativeMetrics, new CometTaskMemoryManager(id), batchSize = COMET_BATCH_SIZE.get(), + use_unified_memory_manager = conf.getBoolean("spark.memory.offHeap.enabled", false), + memory_limit = CometSparkSessionExtensions.getCometMemoryOverhead(conf), + memory_fraction = COMET_EXEC_MEMORY_FRACTION.get(), debug = COMET_DEBUG_ENABLED.get(), explain = COMET_EXPLAIN_NATIVE_ENABLED.get(), workerThreads = COMET_WORKER_THREADS.get(), diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 61c45daff0..8bff6b5fbd 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -53,7 +53,7 @@ import org.apache.spark.sql.types.{DoubleType, FloatType} import org.apache.comet.CometConf._ import org.apache.comet.CometExplainInfo.getActualPlan -import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSpark34Plus, isSpark40Plus, shouldApplySparkToColumnar, withInfo, withInfos} +import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isOffHeapEnabled, isSpark34Plus, isSpark40Plus, isTesting, shouldApplySparkToColumnar, withInfo, withInfos} import org.apache.comet.parquet.{CometParquetScan, SupportsComet} import org.apache.comet.rules.RewriteJoin import org.apache.comet.serde.OperatorOuterClass.Operator @@ -921,8 +921,9 @@ class CometSparkSessionExtensions override def apply(plan: SparkPlan): SparkPlan = { // Comet required off-heap memory to be enabled - if ("true" != conf.getConfString("spark.memory.offHeap.enabled", "false")) { - logInfo("Comet extension disabled because spark.memory.offHeap.enabled=false") + if (!isOffHeapEnabled(conf) && !isTesting) { + logWarning("Comet native exec disabled because spark.memory.offHeap.enabled=false") + withInfo(plan, "Comet native exec disabled because spark.memory.offHeap.enabled=false") return plan } @@ -1174,8 +1175,7 @@ object CometSparkSessionExtensions extends Logging { } private[comet] def isOffHeapEnabled(conf: SQLConf): Boolean = - conf.contains("spark.memory.offHeap.enabled") && - conf.getConfString("spark.memory.offHeap.enabled").toBoolean + conf.getConfString("spark.memory.offHeap.enabled", "false").toBoolean // Copied from org.apache.spark.util.Utils which is private to Spark. private[comet] def isTesting: Boolean = { diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 64ada91adc..083c0f2b50 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -43,6 +43,7 @@ class Native extends NativeBase { * @return * the address to native query plan. */ + // scalastyle:off @native def createPlan( id: Long, iterators: Array[CometBatchIterator], @@ -50,10 +51,14 @@ class Native extends NativeBase { metrics: CometMetricNode, taskMemoryManager: CometTaskMemoryManager, batchSize: Int, + use_unified_memory_manager: Boolean, + memory_limit: Long, + memory_fraction: Double, debug: Boolean, explain: Boolean, workerThreads: Int, blockingThreads: Int): Long + // scalastyle:on /** * Execute a native query plan based on given input Arrow arrays. From e297d23bd38bc306c90ed21a154d1495f985683e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Dec 2024 10:50:07 -0700 Subject: [PATCH 38/47] feat: Improve shuffle metrics (second attempt) (#1175) * improve shuffle metrics * docs * more metrics * refactor * address feedback --- docs/source/index.rst | 1 + docs/source/user-guide/metrics.md | 66 +++++ docs/source/user-guide/tuning.md | 25 -- native/core/src/execution/shuffle/row.rs | 6 +- .../src/execution/shuffle/shuffle_writer.rs | 271 +++++++++++------- .../sql/comet/CometCollectLimitExec.scala | 3 +- .../spark/sql/comet/CometMetricNode.scala | 11 + .../CometTakeOrderedAndProjectExec.scala | 3 +- .../shuffle/CometShuffleExchangeExec.scala | 15 +- 9 files changed, 261 insertions(+), 140 deletions(-) create mode 100644 docs/source/user-guide/metrics.md diff --git a/docs/source/index.rst b/docs/source/index.rst index 39ad27a57c..21ec36ca95 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -51,6 +51,7 @@ as a native runtime to achieve improvement in terms of query efficiency and quer Configuration Settings Compatibility Guide Tuning Guide + Metrics Guide .. _toc.contributor-guide-links: .. toctree:: diff --git a/docs/source/user-guide/metrics.md b/docs/source/user-guide/metrics.md new file mode 100644 index 0000000000..509d0ae8c0 --- /dev/null +++ b/docs/source/user-guide/metrics.md @@ -0,0 +1,66 @@ + + +# Comet Metrics + +## Spark SQL Metrics + +Set `spark.comet.metrics.detailed=true` to see all available Comet metrics. + +### CometScanExec + +| Metric | Description | +| ----------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| `scan time` | Total time to scan a Parquet file. This is not comparable to the same metric in Spark because Comet's scan metric is more accurate. Although both Comet and Spark measure the time in nanoseconds, Spark rounds this time to the nearest millisecond per batch and Comet does not. | + +### Exchange + +Comet adds some additional metrics: + +| Metric | Description | +| ------------------------------- | ------------------------------------------------------------- | +| `native shuffle time` | Total time in native code excluding any child operators. | +| `repartition time` | Time to repartition batches. | +| `memory pool time` | Time interacting with memory pool. | +| `encoding and compression time` | Time to encode batches in IPC format and compress using ZSTD. | + +## Native Metrics + +Setting `spark.comet.explain.native.enabled=true` will cause native plans to be logged in each executor. Metrics are +logged for each native plan (and there is one plan per task, so this is very verbose). + +Here is a guide to some of the native metrics. + +### ScanExec + +| Metric | Description | +| ----------------- | --------------------------------------------------------------------------------------------------- | +| `elapsed_compute` | Total time spent in this operator, fetching batches from a JVM iterator. | +| `jvm_fetch_time` | Time spent in the JVM fetching input batches to be read by this `ScanExec` instance. | +| `arrow_ffi_time` | Time spent using Arrow FFI to create Arrow batches from the memory addresses returned from the JVM. | + +### ShuffleWriterExec + +| Metric | Description | +| ----------------- | ------------------------------------------------------------- | +| `elapsed_compute` | Total time excluding any child operators. | +| `repart_time` | Time to repartition batches. | +| `ipc_time` | Time to encode batches in IPC format and compress using ZSTD. | +| `mempool_time` | Time interacting with memory pool. | +| `write_time` | Time spent writing bytes to disk. | diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index f10a0dde9e..d68481d172 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -103,31 +103,6 @@ native shuffle currently only supports `HashPartitioning` and `SinglePartitionin To enable native shuffle, set `spark.comet.exec.shuffle.mode` to `native`. If this mode is explicitly set, then any shuffle operations that cannot be supported in this mode will fall back to Spark. -## Metrics - -### Spark SQL Metrics - -Some Comet metrics are not directly comparable to Spark metrics in some cases: - -- `CometScanExec` uses nanoseconds for total scan time. Spark also measures scan time in nanoseconds but converts to - milliseconds _per batch_ which can result in a large loss of precision, making it difficult to compare scan times - between Spark and Comet. - -### Native Metrics - -Setting `spark.comet.explain.native.enabled=true` will cause native plans to be logged in each executor. Metrics are -logged for each native plan (and there is one plan per task, so this is very verbose). - -Here is a guide to some of the native metrics. - -### ScanExec - -| Metric | Description | -| ----------------- | --------------------------------------------------------------------------------------------------- | -| `elapsed_compute` | Total time spent in this operator, fetching batches from a JVM iterator. | -| `jvm_fetch_time` | Time spent in the JVM fetching input batches to be read by this `ScanExec` instance. | -| `arrow_ffi_time` | Time spent using Arrow FFI to create Arrow batches from the memory addresses returned from the JVM. | - ## Explain Plan ### Extended Explain With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index ce752e68af..ecab77d966 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -40,6 +40,7 @@ use arrow_array::{ Array, ArrayRef, RecordBatch, RecordBatchOptions, }; use arrow_schema::{ArrowError, DataType, Field, Schema, TimeUnit}; +use datafusion::physical_plan::metrics::Time; use jni::sys::{jint, jlong}; use std::{ fs::OpenOptions, @@ -3354,7 +3355,10 @@ pub fn process_sorted_row_partition( let mut frozen: Vec = vec![]; let mut cursor = Cursor::new(&mut frozen); cursor.seek(SeekFrom::End(0))?; - written += write_ipc_compressed(&batch, &mut cursor)?; + + // we do not collect metrics in Native_writeSortedFileNative + let ipc_time = Time::default(); + written += write_ipc_compressed(&batch, &mut cursor, &ipc_time)?; if let Some(checksum) = &mut current_checksum { checksum.update(&mut cursor)?; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 7587ff06dc..fcc8c51f60 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,22 +17,14 @@ //! Defines the External shuffle repartition plan. -use std::{ - any::Any, - fmt, - fmt::{Debug, Formatter}, - fs::{File, OpenOptions}, - io::{BufReader, BufWriter, Cursor, Read, Seek, SeekFrom, Write}, - path::Path, - sync::Arc, - task::{Context, Poll}, +use crate::{ + common::bit::ceil, + errors::{CometError, CometResult}, }; - use arrow::{datatypes::*, ipc::writer::StreamWriter}; use async_trait::async_trait; use bytes::Buf; use crc32fast::Hasher; -use datafusion::physical_plan::metrics::Time; use datafusion::{ arrow::{ array::*, @@ -48,23 +40,32 @@ use datafusion::{ runtime_env::RuntimeEnv, }, physical_plan::{ - metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, + metrics::{ + BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, + }, stream::RecordBatchStreamAdapter, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }, }; +use datafusion_comet_spark_expr::spark_hash::create_murmur3_hashes; use datafusion_physical_expr::EquivalenceProperties; use futures::executor::block_on; use futures::{lock::Mutex, Stream, StreamExt, TryFutureExt, TryStreamExt}; use itertools::Itertools; use simd_adler32::Adler32; - -use crate::{ - common::bit::ceil, - errors::{CometError, CometResult}, +use std::io::Error; +use std::{ + any::Any, + fmt, + fmt::{Debug, Formatter}, + fs::{File, OpenOptions}, + io::{BufReader, BufWriter, Cursor, Read, Seek, SeekFrom, Write}, + path::Path, + sync::Arc, + task::{Context, Poll}, }; -use datafusion_comet_spark_expr::spark_hash::create_murmur3_hashes; +use tokio::time::Instant; /// The status of appending rows to a partition buffer. enum AppendRowStatus { @@ -242,7 +243,7 @@ impl PartitionBuffer { /// Initializes active builders if necessary. /// Returns error if memory reservation fails. - fn init_active_if_necessary(&mut self) -> Result { + fn init_active_if_necessary(&mut self, metrics: &ShuffleRepartitionerMetrics) -> Result { let mut mem_diff = 0; if self.active.is_empty() { @@ -256,9 +257,13 @@ impl PartitionBuffer { .sum::(); } + let mut mempool_timer = metrics.mempool_time.timer(); self.reservation.try_grow(self.active_slots_mem_size)?; + mempool_timer.stop(); + let mut repart_timer = metrics.repart_time.timer(); self.active = new_array_builders(&self.schema, self.batch_size); + repart_timer.stop(); mem_diff += self.active_slots_mem_size as isize; } @@ -271,13 +276,13 @@ impl PartitionBuffer { columns: &[ArrayRef], indices: &[usize], start_index: usize, - time_metric: &Time, + metrics: &ShuffleRepartitionerMetrics, ) -> AppendRowStatus { let mut mem_diff = 0; let mut start = start_index; // lazy init because some partition may be empty - let init = self.init_active_if_necessary(); + let init = self.init_active_if_necessary(metrics); if init.is_err() { return AppendRowStatus::StartIndex(start); } @@ -285,6 +290,8 @@ impl PartitionBuffer { while start < indices.len() { let end = (start + self.batch_size).min(indices.len()); + + let mut repart_timer = metrics.repart_time.timer(); self.active .iter_mut() .zip(columns) @@ -292,16 +299,16 @@ impl PartitionBuffer { append_columns(builder, column, &indices[start..end], column.data_type()); }); self.num_active_rows += end - start; + repart_timer.stop(); + if self.num_active_rows >= self.batch_size { - let mut timer = time_metric.timer(); - let flush = self.flush(); + let flush = self.flush(&metrics.ipc_time); if let Err(e) = flush { return AppendRowStatus::MemDiff(Err(e)); } mem_diff += flush.unwrap(); - timer.stop(); - let init = self.init_active_if_necessary(); + let init = self.init_active_if_necessary(metrics); if init.is_err() { return AppendRowStatus::StartIndex(end); } @@ -313,7 +320,7 @@ impl PartitionBuffer { } /// flush active data into frozen bytes - fn flush(&mut self) -> Result { + fn flush(&mut self, ipc_time: &Time) -> Result { if self.num_active_rows == 0 { return Ok(0); } @@ -330,7 +337,7 @@ impl PartitionBuffer { let frozen_capacity_old = self.frozen.capacity(); let mut cursor = Cursor::new(&mut self.frozen); cursor.seek(SeekFrom::End(0))?; - write_ipc_compressed(&frozen_batch, &mut cursor)?; + write_ipc_compressed(&frozen_batch, &mut cursor, ipc_time)?; mem_diff += (self.frozen.capacity() - frozen_capacity_old) as isize; Ok(mem_diff) @@ -628,6 +635,21 @@ struct ShuffleRepartitionerMetrics { /// metrics baseline: BaselineMetrics, + /// Time to perform repartitioning + repart_time: Time, + + /// Time interacting with memory pool + mempool_time: Time, + + /// Time encoding batches to IPC format + ipc_time: Time, + + /// Time spent writing to disk. Maps to "shuffleWriteTime" in Spark SQL Metrics. + write_time: Time, + + /// Number of input batches + input_batches: Count, + /// count of spills during the execution of the operator spill_count: Count, @@ -642,6 +664,11 @@ impl ShuffleRepartitionerMetrics { fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { Self { baseline: BaselineMetrics::new(metrics, partition), + repart_time: MetricBuilder::new(metrics).subset_time("repart_time", partition), + mempool_time: MetricBuilder::new(metrics).subset_time("mempool_time", partition), + ipc_time: MetricBuilder::new(metrics).subset_time("ipc_time", partition), + write_time: MetricBuilder::new(metrics).subset_time("write_time", partition), + input_batches: MetricBuilder::new(metrics).counter("input_batches", partition), spill_count: MetricBuilder::new(metrics).spill_count(partition), spilled_bytes: MetricBuilder::new(metrics).spilled_bytes(partition), data_size: MetricBuilder::new(metrics).counter("data_size", partition), @@ -701,6 +728,7 @@ impl ShuffleRepartitioner { /// This function will slice input batch according to configured batch size and then /// shuffle rows into corresponding partition buffer. async fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { + let start_time = Instant::now(); let mut start = 0; while start < batch.num_rows() { let end = (start + self.batch_size).min(batch.num_rows()); @@ -708,6 +736,11 @@ impl ShuffleRepartitioner { self.partitioning_batch(batch).await?; start = end; } + self.metrics.input_batches.add(1); + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); Ok(()) } @@ -738,53 +771,61 @@ impl ShuffleRepartitioner { let num_output_partitions = self.num_output_partitions; match &self.partitioning { Partitioning::Hash(exprs, _) => { - let arrays = exprs - .iter() - .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) - .collect::>>()?; - - // use identical seed as spark hash partition - let hashes_buf = &mut self.hashes_buf[..arrays[0].len()]; - hashes_buf.fill(42_u32); - - // Hash arrays and compute buckets based on number of partitions - let partition_ids = &mut self.partition_ids[..arrays[0].len()]; - create_murmur3_hashes(&arrays, hashes_buf)? - .iter() - .enumerate() - .for_each(|(idx, hash)| { - partition_ids[idx] = pmod(*hash, num_output_partitions) as u64 + let (partition_starts, shuffled_partition_ids): (Vec, Vec) = { + let mut timer = self.metrics.repart_time.timer(); + let arrays = exprs + .iter() + .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) + .collect::>>()?; + + // use identical seed as spark hash partition + let hashes_buf = &mut self.hashes_buf[..arrays[0].len()]; + hashes_buf.fill(42_u32); + + // Hash arrays and compute buckets based on number of partitions + let partition_ids = &mut self.partition_ids[..arrays[0].len()]; + create_murmur3_hashes(&arrays, hashes_buf)? + .iter() + .enumerate() + .for_each(|(idx, hash)| { + partition_ids[idx] = pmod(*hash, num_output_partitions) as u64 + }); + + // count each partition size + let mut partition_counters = vec![0usize; num_output_partitions]; + partition_ids + .iter() + .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); + + // accumulate partition counters into partition ends + // e.g. partition counter: [1, 3, 2, 1] => [1, 4, 6, 7] + let mut partition_ends = partition_counters; + let mut accum = 0; + partition_ends.iter_mut().for_each(|v| { + *v += accum; + accum = *v; }); - // count each partition size - let mut partition_counters = vec![0usize; num_output_partitions]; - partition_ids - .iter() - .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - - // accumulate partition counters into partition ends - // e.g. partition counter: [1, 3, 2, 1] => [1, 4, 6, 7] - let mut partition_ends = partition_counters; - let mut accum = 0; - partition_ends.iter_mut().for_each(|v| { - *v += accum; - accum = *v; - }); - - // calculate shuffled partition ids - // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] => [6, 1, 2, 3, 4, 5, 0] which is the - // row indices for rows ordered by their partition id. For example, first partition - // 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. - let mut shuffled_partition_ids = vec![0usize; input.num_rows()]; - for (index, partition_id) in partition_ids.iter().enumerate().rev() { - partition_ends[*partition_id as usize] -= 1; - let end = partition_ends[*partition_id as usize]; - shuffled_partition_ids[end] = index; - } + // calculate shuffled partition ids + // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] => [6, 1, 2, 3, 4, 5, 0] which is the + // row indices for rows ordered by their partition id. For example, first partition + // 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. + let mut shuffled_partition_ids = vec![0usize; input.num_rows()]; + for (index, partition_id) in partition_ids.iter().enumerate().rev() { + partition_ends[*partition_id as usize] -= 1; + let end = partition_ends[*partition_id as usize]; + shuffled_partition_ids[end] = index; + } - // after calculating, partition ends become partition starts - let mut partition_starts = partition_ends; - partition_starts.push(input.num_rows()); + // after calculating, partition ends become partition starts + let mut partition_starts = partition_ends; + partition_starts.push(input.num_rows()); + timer.stop(); + Ok::<(Vec, Vec), DataFusionError>(( + partition_starts, + shuffled_partition_ids, + )) + }?; // For each interval of row indices of partition, taking rows from input batch and // appending into output buffer. @@ -804,11 +845,20 @@ impl ShuffleRepartitioner { if mem_diff > 0 { let mem_increase = mem_diff as usize; - if self.reservation.try_grow(mem_increase).is_err() { + + let try_grow = { + let mut mempool_timer = self.metrics.mempool_time.timer(); + let result = self.reservation.try_grow(mem_increase); + mempool_timer.stop(); + result + }; + + if try_grow.is_err() { self.spill().await?; + let mut mempool_timer = self.metrics.mempool_time.timer(); self.reservation.free(); self.reservation.try_grow(mem_increase)?; - + mempool_timer.stop(); mem_diff = 0; } } @@ -816,7 +866,9 @@ impl ShuffleRepartitioner { if mem_diff < 0 { let mem_used = self.reservation.size(); let mem_decrease = mem_used.min(-mem_diff as usize); + let mut mempool_timer = self.metrics.mempool_time.timer(); self.reservation.shrink(mem_decrease); + mempool_timer.stop(); } } } @@ -848,12 +900,13 @@ impl ShuffleRepartitioner { /// Writes buffered shuffled record batches into Arrow IPC bytes. async fn shuffle_write(&mut self) -> Result { + let mut elapsed_compute = self.metrics.baseline.elapsed_compute().timer(); let num_output_partitions = self.num_output_partitions; let buffered_partitions = &mut self.buffered_partitions; let mut output_batches: Vec> = vec![vec![]; num_output_partitions]; - + let mut offsets = vec![0; num_output_partitions + 1]; for i in 0..num_output_partitions { - buffered_partitions[i].flush()?; + buffered_partitions[i].flush(&self.metrics.ipc_time)?; output_batches[i] = std::mem::take(&mut buffered_partitions[i].frozen); } @@ -863,53 +916,38 @@ impl ShuffleRepartitioner { let data_file = self.output_data_file.clone(); let index_file = self.output_index_file.clone(); - let mut offsets = vec![0; num_output_partitions + 1]; - let mut output_data = OpenOptions::new() + let mut write_time = self.metrics.write_time.timer(); + + let output_data = OpenOptions::new() .write(true) .create(true) .truncate(true) .open(data_file) .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {:?}", e)))?; - for i in 0..num_output_partitions { - let mut timer = self.metrics.baseline.elapsed_compute().timer(); + let mut output_data = BufWriter::new(output_data); + for i in 0..num_output_partitions { offsets[i] = output_data.stream_position()?; output_data.write_all(&output_batches[i])?; - - timer.stop(); - output_batches[i].clear(); // append partition in each spills for spill in &output_spills { let length = spill.offsets[i + 1] - spill.offsets[i]; if length > 0 { - let mut timer = self.metrics.baseline.elapsed_compute().timer(); - let mut spill_file = - BufReader::new(File::open(spill.file.path()).map_err(|e| { - DataFusionError::Execution(format!("shuffle write error: {:?}", e)) - })?); + BufReader::new(File::open(spill.file.path()).map_err(Self::to_df_err)?); spill_file.seek(SeekFrom::Start(spill.offsets[i]))?; - std::io::copy(&mut spill_file.take(length), &mut output_data).map_err(|e| { - DataFusionError::Execution(format!("shuffle write error: {:?}", e)) - })?; - - timer.stop(); + std::io::copy(&mut spill_file.take(length), &mut output_data) + .map_err(Self::to_df_err)?; } } } - let mut timer = self.metrics.baseline.elapsed_compute().timer(); output_data.flush()?; - timer.stop(); // add one extra offset at last to ease partition length computation - offsets[num_output_partitions] = output_data - .stream_position() - .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {:?}", e)))?; - - let mut timer = self.metrics.baseline.elapsed_compute().timer(); + offsets[num_output_partitions] = output_data.stream_position().map_err(Self::to_df_err)?; let mut output_index = BufWriter::new(File::create(index_file).map_err(|e| { @@ -918,19 +956,27 @@ impl ShuffleRepartitioner { for offset in offsets { output_index .write_all(&(offset as i64).to_le_bytes()[..]) - .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {:?}", e)))?; + .map_err(Self::to_df_err)?; } output_index.flush()?; - timer.stop(); + write_time.stop(); + let mut mempool_timer = self.metrics.mempool_time.timer(); let used = self.reservation.size(); self.reservation.shrink(used); + mempool_timer.stop(); + + elapsed_compute.stop(); // shuffle writer always has empty output Ok(Box::pin(EmptyStream::try_new(Arc::clone(&self.schema))?)) } + fn to_df_err(e: Error) -> DataFusionError { + DataFusionError::Execution(format!("shuffle write error: {:?}", e)) + } + fn used(&self) -> usize { self.reservation.size() } @@ -959,7 +1005,7 @@ impl ShuffleRepartitioner { return Ok(0); } - let mut timer = self.metrics.baseline.elapsed_compute().timer(); + let mut timer = self.metrics.write_time.timer(); let spillfile = self .runtime @@ -969,6 +1015,7 @@ impl ShuffleRepartitioner { &mut self.buffered_partitions, spillfile.path(), self.num_output_partitions, + &self.metrics.ipc_time, )?; timer.stop(); @@ -995,12 +1042,10 @@ impl ShuffleRepartitioner { let output = &mut self.buffered_partitions[partition_id]; - let time_metric = self.metrics.baseline.elapsed_compute(); - // If the range of indices is not big enough, just appending the rows into // active array builders instead of directly adding them as a record batch. let mut start_index: usize = 0; - let mut output_ret = output.append_rows(columns, indices, start_index, time_metric); + let mut output_ret = output.append_rows(columns, indices, start_index, &self.metrics); loop { match output_ret { @@ -1012,15 +1057,15 @@ impl ShuffleRepartitioner { // Cannot allocate enough memory for the array builders in the partition, // spill partitions and retry. self.spill().await?; - self.reservation.free(); + let mut mempool_timer = self.metrics.mempool_time.timer(); + self.reservation.free(); let output = &mut self.buffered_partitions[partition_id]; output.reservation.free(); - - let time_metric = self.metrics.baseline.elapsed_compute(); + mempool_timer.stop(); start_index = new_start; - output_ret = output.append_rows(columns, indices, start_index, time_metric); + output_ret = output.append_rows(columns, indices, start_index, &self.metrics); if let AppendRowStatus::StartIndex(new_start) = output_ret { if new_start == start_index { @@ -1045,11 +1090,12 @@ fn spill_into( buffered_partitions: &mut [PartitionBuffer], path: &Path, num_output_partitions: usize, + ipc_time: &Time, ) -> Result> { let mut output_batches: Vec> = vec![vec![]; num_output_partitions]; for i in 0..num_output_partitions { - buffered_partitions[i].flush()?; + buffered_partitions[i].flush(ipc_time)?; output_batches[i] = std::mem::take(&mut buffered_partitions[i].frozen); } let path = path.to_owned(); @@ -1485,10 +1531,13 @@ impl Checksum { pub(crate) fn write_ipc_compressed( batch: &RecordBatch, output: &mut W, + ipc_time: &Time, ) -> Result { if batch.num_rows() == 0 { return Ok(0); } + + let mut timer = ipc_time.timer(); let start_pos = output.stream_position()?; // write ipc_length placeholder @@ -1508,8 +1557,10 @@ pub(crate) fn write_ipc_compressed( // fill ipc length output.seek(SeekFrom::Start(start_pos))?; output.write_all(&ipc_length.to_le_bytes()[..])?; - output.seek(SeekFrom::Start(end_pos))?; + + timer.stop(); + Ok((end_pos - start_pos) as usize) } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index 8ea0b17654..f75af5076b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -57,7 +57,8 @@ case class CometCollectLimitExec( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), "numPartitions" -> SQLMetrics.createMetric( sparkContext, - "number of partitions")) ++ readMetrics ++ writeMetrics + "number of partitions")) ++ readMetrics ++ writeMetrics ++ CometMetricNode.shuffleMetrics( + sparkContext) private lazy val serializer: Serializer = new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala index 47c89d9433..a26fa28c8b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala @@ -130,6 +130,17 @@ object CometMetricNode { "spilled_rows" -> SQLMetrics.createMetric(sc, "Total spilled rows")) } + def shuffleMetrics(sc: SparkContext): Map[String, SQLMetric] = { + Map( + "elapsed_compute" -> SQLMetrics.createNanoTimingMetric(sc, "native shuffle time"), + "mempool_time" -> SQLMetrics.createNanoTimingMetric(sc, "memory pool time"), + "repart_time" -> SQLMetrics.createNanoTimingMetric(sc, "repartition time"), + "ipc_time" -> SQLMetrics.createNanoTimingMetric(sc, "encoding and compression time"), + "spill_count" -> SQLMetrics.createMetric(sc, "number of spills"), + "spilled_bytes" -> SQLMetrics.createMetric(sc, "spilled bytes"), + "input_batches" -> SQLMetrics.createMetric(sc, "number of input batches")) + } + /** * Creates a [[CometMetricNode]] from a [[CometPlan]]. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 5582f4d687..19586628a7 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -57,7 +57,8 @@ case class CometTakeOrderedAndProjectExec( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), "numPartitions" -> SQLMetrics.createMetric( sparkContext, - "number of partitions")) ++ readMetrics ++ writeMetrics + "number of partitions")) ++ readMetrics ++ writeMetrics ++ CometMetricNode.shuffleMetrics( + sparkContext) private lazy val serializer: Serializer = new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index b1dd9ac836..0cd8a9ce67 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -79,7 +79,8 @@ case class CometShuffleExchangeExec( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), "numPartitions" -> SQLMetrics.createMetric( sparkContext, - "number of partitions")) ++ readMetrics ++ writeMetrics + "number of partitions")) ++ readMetrics ++ writeMetrics ++ CometMetricNode.shuffleMetrics( + sparkContext) override def nodeName: String = if (shuffleType == CometNativeShuffle) { "CometExchange" @@ -477,11 +478,21 @@ class CometShuffleWriteProcessor( // Call native shuffle write val nativePlan = getNativePlan(tempDataFilename, tempIndexFilename) + val detailedMetrics = Seq( + "elapsed_compute", + "ipc_time", + "repart_time", + "mempool_time", + "input_batches", + "spill_count", + "spilled_bytes") + // Maps native metrics to SQL metrics val nativeSQLMetrics = Map( "output_rows" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN), "data_size" -> metrics("dataSize"), - "elapsed_compute" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME)) + "write_time" -> metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME)) ++ + metrics.filterKeys(detailedMetrics.contains) val nativeMetrics = CometMetricNode(nativeSQLMetrics) // Getting rid of the fake partitionId From 8f4a8a54a292d86a30fb66d3dac442bff46a1039 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 19 Dec 2024 13:06:56 -0800 Subject: [PATCH 39/47] fix: stddev_pop should not directly return 0.0 when count is 1.0 (#1184) * add test * fix * fix * fix --- native/spark-expr/src/variance.rs | 6 ++---- .../apache/comet/exec/CometAggregateSuite.scala | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index 2f4d8091c2..4370d89ff4 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -245,10 +245,8 @@ impl Accumulator for VarianceAccumulator { Ok(ScalarValue::Float64(match self.count { count if count == 0.0 => None, - count if count == 1.0 => { - if let StatsType::Population = self.stats_type { - Some(0.0) - } else if self.null_on_divide_by_zero { + count if count == 1.0 && StatsType::Sample == self.stats_type => { + if self.null_on_divide_by_zero { None } else { Some(f64::NAN) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index e337d36c48..9a642f12f7 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -38,6 +38,23 @@ import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ + test("stddev_pop should return NaN for some cases") { + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_EXPR_STDDEV_ENABLED.key -> "true") { + Seq(true, false).foreach { nullOnDivideByZero => + withSQLConf("spark.sql.legacy.statisticalAggregate" -> nullOnDivideByZero.toString) { + + val data: Seq[(Float, Int)] = Seq((Float.PositiveInfinity, 1)) + withParquetTable(data, "tbl", false) { + val df = sql("SELECT stddev_pop(_1), stddev_pop(_2) FROM tbl") + checkSparkAnswer(df) + } + } + } + } + } + test("count with aggregation filter") { withSQLConf( CometConf.COMET_ENABLED.key -> "true", From ea6d20511e813a2698c47a964b3a0739e9543add Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 20 Dec 2024 11:11:14 -0700 Subject: [PATCH 40/47] feat: Make native shuffle compression configurable and respect `spark.shuffle.compress` (#1185) * Make shuffle compression codec and level configurable * remove lz4 references * docs * update comment * clippy * fix benches * clippy * clippy * disable test for miri * remove lz4 reference from proto --- .../scala/org/apache/comet/CometConf.scala | 14 ++- .../shuffle/IpcInputStreamIterator.scala | 6 +- .../execution/shuffle/ShuffleUtils.scala | 31 ++++--- docs/source/user-guide/configs.md | 3 +- docs/source/user-guide/tuning.md | 6 ++ native/core/benches/shuffle_writer.rs | 87 +++++++++++++----- native/core/src/execution/planner.rs | 17 +++- native/core/src/execution/shuffle/mod.rs | 2 +- native/core/src/execution/shuffle/row.rs | 5 +- .../src/execution/shuffle/shuffle_writer.rs | 90 +++++++++++++++---- native/proto/src/proto/operator.proto | 7 ++ .../shuffle/CometShuffleExchangeExec.scala | 14 ++- .../shuffle/CometShuffleManager.scala | 2 +- 13 files changed, 221 insertions(+), 63 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index b602d7cf1c..8815ac4eb8 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -272,13 +272,21 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) - val COMET_EXEC_SHUFFLE_CODEC: ConfigEntry[String] = conf( - s"$COMET_EXEC_CONFIG_PREFIX.shuffle.codec") + val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf( + s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") .doc( - "The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported.") + "The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. " + + "Compression can be disabled by setting spark.shuffle.compress=false.") .stringConf + .checkValues(Set("zstd")) .createWithDefault("zstd") + val COMET_EXEC_SHUFFLE_COMPRESSION_LEVEL: ConfigEntry[Int] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.level") + .doc("The compression level to use when compression shuffle files.") + .intConf + .createWithDefault(1) + val COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.columnar.shuffle.async.enabled") .doc("Whether to enable asynchronous shuffle for Arrow-based shuffle.") diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala index 281c481083..d1d5af350f 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala @@ -110,8 +110,10 @@ case class IpcInputStreamIterator( currentLimitedInputStream = is if (decompressingNeeded) { - val zs = ShuffleUtils.compressionCodecForShuffling.compressedInputStream(is) - Channels.newChannel(zs) + ShuffleUtils.compressionCodecForShuffling match { + case Some(codec) => Channels.newChannel(codec.compressedInputStream(is)) + case _ => Channels.newChannel(is) + } } else { Channels.newChannel(is) } diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala index eea134ab5e..23b4a5ec23 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala @@ -21,22 +21,33 @@ package org.apache.spark.sql.comet.execution.shuffle import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.IO_COMPRESSION_CODEC +import org.apache.spark.internal.config.{IO_COMPRESSION_CODEC, SHUFFLE_COMPRESS} import org.apache.spark.io.CompressionCodec -import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf private[spark] object ShuffleUtils extends Logging { - lazy val compressionCodecForShuffling: CompressionCodec = { + // optional compression codec to use when compressing shuffle files + lazy val compressionCodecForShuffling: Option[CompressionCodec] = { val sparkConf = SparkEnv.get.conf - val codecName = CometConf.COMET_EXEC_SHUFFLE_CODEC.get(SQLConf.get) - - // only zstd compression is supported at the moment - if (codecName != "zstd") { - logWarning( - s"Overriding config ${IO_COMPRESSION_CODEC}=${codecName} in shuffling, force using zstd") + val shuffleCompressionEnabled = sparkConf.getBoolean(SHUFFLE_COMPRESS.key, true) + val sparkShuffleCodec = sparkConf.get(IO_COMPRESSION_CODEC.key, "lz4") + val cometShuffleCodec = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get() + if (shuffleCompressionEnabled) { + if (sparkShuffleCodec != cometShuffleCodec) { + logWarning( + s"Overriding config $IO_COMPRESSION_CODEC=$sparkShuffleCodec in shuffling, " + + s"force using $cometShuffleCodec") + } + cometShuffleCodec match { + case "zstd" => + Some(CompressionCodec.createCodec(sparkConf, "zstd")) + case other => + throw new UnsupportedOperationException( + s"Unsupported shuffle compression codec: $other") + } + } else { + None } - CompressionCodec.createCodec(sparkConf, "zstd") } } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 69da792223..7881f07632 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -50,7 +50,8 @@ Comet provides the following configuration settings. | spark.comet.exec.memoryFraction | The fraction of memory from Comet memory overhead that the native memory manager can use for execution. The purpose of this config is to set aside memory for untracked data structures, as well as imprecise size estimation during memory acquisition. | 0.7 | | spark.comet.exec.project.enabled | Whether to enable project by default. | true | | spark.comet.exec.replaceSortMergeJoin | Experimental feature to force Spark to replace SortMergeJoin with ShuffledHashJoin for improved performance. This feature is not stable yet. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | false | -| spark.comet.exec.shuffle.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. | zstd | +| spark.comet.exec.shuffle.compression.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. Compression can be disabled by setting spark.shuffle.compress=false. | zstd | +| spark.comet.exec.shuffle.compression.level | The compression level to use when compression shuffle files. | 1 | | spark.comet.exec.shuffle.enabled | Whether to enable Comet native shuffle. Note that this requires setting 'spark.shuffle.manager' to 'org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager'. 'spark.shuffle.manager' must be set before starting the Spark application and cannot be changed during the application. | true | | spark.comet.exec.sort.enabled | Whether to enable sort by default. | true | | spark.comet.exec.sortMergeJoin.enabled | Whether to enable sortMergeJoin by default. | true | diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index d68481d172..e04e750b47 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -103,6 +103,12 @@ native shuffle currently only supports `HashPartitioning` and `SinglePartitionin To enable native shuffle, set `spark.comet.exec.shuffle.mode` to `native`. If this mode is explicitly set, then any shuffle operations that cannot be supported in this mode will fall back to Spark. +### Shuffle Compression + +By default, Spark compresses shuffle files using LZ4 compression. Comet overrides this behavior with ZSTD compression. +Compression can be disabled by setting `spark.shuffle.compress=false`, which may result in faster shuffle times in +certain environments, such as single-node setups with fast NVMe drives, at the expense of increased disk space usage. + ## Explain Plan ### Extended Explain With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 272887238e..865ca73b4a 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -15,36 +15,47 @@ // specific language governing permissions and limitations // under the License. +use arrow_array::builder::Int32Builder; use arrow_array::{builder::StringBuilder, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; -use comet::execution::shuffle::ShuffleWriterExec; +use comet::execution::shuffle::{write_ipc_compressed, CompressionCodec, ShuffleWriterExec}; use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::physical_plan::metrics::Time; use datafusion::{ physical_plan::{common::collect, memory::MemoryExec, ExecutionPlan}, prelude::SessionContext, }; use datafusion_physical_expr::{expressions::Column, Partitioning}; +use std::io::Cursor; use std::sync::Arc; use tokio::runtime::Runtime; fn criterion_benchmark(c: &mut Criterion) { - let batch = create_batch(); - let mut batches = Vec::new(); - for _ in 0..10 { - batches.push(batch.clone()); - } - let partitions = &[batches]; - let exec = ShuffleWriterExec::try_new( - Arc::new(MemoryExec::try_new(partitions, batch.schema(), None).unwrap()), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), - "/tmp/data.out".to_string(), - "/tmp/index.out".to_string(), - ) - .unwrap(); - let mut group = c.benchmark_group("shuffle_writer"); - group.bench_function("shuffle_writer", |b| { + group.bench_function("shuffle_writer: encode (no compression))", |b| { + let batch = create_batch(8192, true); + let mut buffer = vec![]; + let mut cursor = Cursor::new(&mut buffer); + let ipc_time = Time::default(); + b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::None, &ipc_time)); + }); + group.bench_function("shuffle_writer: encode and compress (zstd level 1)", |b| { + let batch = create_batch(8192, true); + let mut buffer = vec![]; + let mut cursor = Cursor::new(&mut buffer); + let ipc_time = Time::default(); + b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::Zstd(1), &ipc_time)); + }); + group.bench_function("shuffle_writer: encode and compress (zstd level 6)", |b| { + let batch = create_batch(8192, true); + let mut buffer = vec![]; + let mut cursor = Cursor::new(&mut buffer); + let ipc_time = Time::default(); + b.iter(|| write_ipc_compressed(&batch, &mut cursor, &CompressionCodec::Zstd(6), &ipc_time)); + }); + group.bench_function("shuffle_writer: end to end", |b| { let ctx = SessionContext::new(); + let exec = create_shuffle_writer_exec(CompressionCodec::Zstd(1)); b.iter(|| { let task_ctx = ctx.task_ctx(); let stream = exec.execute(0, task_ctx).unwrap(); @@ -54,19 +65,47 @@ fn criterion_benchmark(c: &mut Criterion) { }); } -fn create_batch() -> RecordBatch { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); +fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWriterExec { + let batches = create_batches(8192, 10); + let schema = batches[0].schema(); + let partitions = &[batches]; + ShuffleWriterExec::try_new( + Arc::new(MemoryExec::try_new(partitions, schema, None).unwrap()), + Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + compression_codec, + "/tmp/data.out".to_string(), + "/tmp/index.out".to_string(), + ) + .unwrap() +} + +fn create_batches(size: usize, count: usize) -> Vec { + let batch = create_batch(size, true); + let mut batches = Vec::new(); + for _ in 0..count { + batches.push(batch.clone()); + } + batches +} + +fn create_batch(num_rows: usize, allow_nulls: bool) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("c0", DataType::Int32, true), + Field::new("c1", DataType::Utf8, true), + ])); + let mut a = Int32Builder::new(); let mut b = StringBuilder::new(); - for i in 0..8192 { - if i % 10 == 0 { + for i in 0..num_rows { + a.append_value(i as i32); + if allow_nulls && i % 10 == 0 { b.append_null(); } else { - b.append_value(format!("{i}")); + b.append_value(format!("this is string number {i}")); } } - let array = b.finish(); - - RecordBatch::try_new(schema.clone(), vec![Arc::new(array)]).unwrap() + let a = a.finish(); + let b = b.finish(); + RecordBatch::try_new(schema.clone(), vec![Arc::new(a), Arc::new(b)]).unwrap() } fn config() -> Criterion { diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 3ac830c04e..0a74933547 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -68,6 +68,7 @@ use datafusion_comet_spark_expr::{create_comet_physical_fun, create_negate_expr} use datafusion_functions_nested::concat::ArrayAppend; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; +use crate::execution::shuffle::CompressionCodec; use crate::execution::spark_plan::SparkPlan; use datafusion_comet_proto::{ spark_expression::{ @@ -76,8 +77,8 @@ use datafusion_comet_proto::{ }, spark_operator::{ self, lower_window_frame_bound::LowerFrameBoundStruct, operator::OpStruct, - upper_window_frame_bound::UpperFrameBoundStruct, BuildSide, JoinType, Operator, - WindowFrameType, + upper_window_frame_bound::UpperFrameBoundStruct, BuildSide, + CompressionCodec as SparkCompressionCodec, JoinType, Operator, WindowFrameType, }, spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; @@ -1049,9 +1050,21 @@ impl PhysicalPlanner { let partitioning = self .create_partitioning(writer.partitioning.as_ref().unwrap(), child.schema())?; + let codec = match writer.codec.try_into() { + Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None), + Ok(SparkCompressionCodec::Zstd) => { + Ok(CompressionCodec::Zstd(writer.compression_level)) + } + _ => Err(ExecutionError::GeneralError(format!( + "Unsupported shuffle compression codec: {:?}", + writer.codec + ))), + }?; + let shuffle_writer = Arc::new(ShuffleWriterExec::try_new( Arc::clone(&child.native_plan), partitioning, + codec, writer.output_data_file.clone(), writer.output_index_file.clone(), )?); diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index 8721ead74b..8111f5eede 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -19,4 +19,4 @@ mod list; mod map; pub mod row; mod shuffle_writer; -pub use shuffle_writer::ShuffleWriterExec; +pub use shuffle_writer::{write_ipc_compressed, CompressionCodec, ShuffleWriterExec}; diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index ecab77d966..405f642163 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -292,6 +292,7 @@ macro_rules! downcast_builder_ref { } // Expose the macro for other modules. +use crate::execution::shuffle::shuffle_writer::CompressionCodec; pub(crate) use downcast_builder_ref; /// Appends field of row to the given struct builder. `dt` is the data type of the field. @@ -3358,7 +3359,9 @@ pub fn process_sorted_row_partition( // we do not collect metrics in Native_writeSortedFileNative let ipc_time = Time::default(); - written += write_ipc_compressed(&batch, &mut cursor, &ipc_time)?; + // compression codec is not configurable for CometBypassMergeSortShuffleWriter + let codec = CompressionCodec::Zstd(1); + written += write_ipc_compressed(&batch, &mut cursor, &codec, &ipc_time)?; if let Some(checksum) = &mut current_checksum { checksum.update(&mut cursor)?; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index fcc8c51f60..01117199eb 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -90,6 +90,7 @@ pub struct ShuffleWriterExec { /// Metrics metrics: ExecutionPlanMetricsSet, cache: PlanProperties, + codec: CompressionCodec, } impl DisplayAs for ShuffleWriterExec { @@ -126,6 +127,7 @@ impl ExecutionPlan for ShuffleWriterExec { 1 => Ok(Arc::new(ShuffleWriterExec::try_new( Arc::clone(&children[0]), self.partitioning.clone(), + self.codec.clone(), self.output_data_file.clone(), self.output_index_file.clone(), )?)), @@ -152,6 +154,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.partitioning.clone(), metrics, context, + self.codec.clone(), ) .map_err(|e| ArrowError::ExternalError(Box::new(e))), ) @@ -181,6 +184,7 @@ impl ShuffleWriterExec { pub fn try_new( input: Arc, partitioning: Partitioning, + codec: CompressionCodec, output_data_file: String, output_index_file: String, ) -> Result { @@ -197,6 +201,7 @@ impl ShuffleWriterExec { output_data_file, output_index_file, cache, + codec, }) } } @@ -217,6 +222,7 @@ struct PartitionBuffer { batch_size: usize, /// Memory reservation for this partition buffer. reservation: MemoryReservation, + codec: CompressionCodec, } impl PartitionBuffer { @@ -225,6 +231,7 @@ impl PartitionBuffer { batch_size: usize, partition_id: usize, runtime: &Arc, + codec: CompressionCodec, ) -> Self { let reservation = MemoryConsumer::new(format!("PartitionBuffer[{}]", partition_id)) .with_can_spill(true) @@ -238,6 +245,7 @@ impl PartitionBuffer { num_active_rows: 0, batch_size, reservation, + codec, } } @@ -337,7 +345,7 @@ impl PartitionBuffer { let frozen_capacity_old = self.frozen.capacity(); let mut cursor = Cursor::new(&mut self.frozen); cursor.seek(SeekFrom::End(0))?; - write_ipc_compressed(&frozen_batch, &mut cursor, ipc_time)?; + write_ipc_compressed(&frozen_batch, &mut cursor, &self.codec, ipc_time)?; mem_diff += (self.frozen.capacity() - frozen_capacity_old) as isize; Ok(mem_diff) @@ -687,6 +695,7 @@ impl ShuffleRepartitioner { metrics: ShuffleRepartitionerMetrics, runtime: Arc, batch_size: usize, + codec: CompressionCodec, ) -> Self { let num_output_partitions = partitioning.partition_count(); let reservation = MemoryConsumer::new(format!("ShuffleRepartitioner[{}]", partition_id)) @@ -709,7 +718,13 @@ impl ShuffleRepartitioner { schema: Arc::clone(&schema), buffered_partitions: (0..num_output_partitions) .map(|partition_id| { - PartitionBuffer::new(Arc::clone(&schema), batch_size, partition_id, &runtime) + PartitionBuffer::new( + Arc::clone(&schema), + batch_size, + partition_id, + &runtime, + codec.clone(), + ) }) .collect::>(), spills: Mutex::new(vec![]), @@ -1129,6 +1144,7 @@ impl Debug for ShuffleRepartitioner { } } +#[allow(clippy::too_many_arguments)] async fn external_shuffle( mut input: SendableRecordBatchStream, partition_id: usize, @@ -1137,6 +1153,7 @@ async fn external_shuffle( partitioning: Partitioning, metrics: ShuffleRepartitionerMetrics, context: Arc, + codec: CompressionCodec, ) -> Result { let schema = input.schema(); let mut repartitioner = ShuffleRepartitioner::new( @@ -1148,6 +1165,7 @@ async fn external_shuffle( metrics, context.runtime_env(), context.session_config().batch_size(), + codec, ); while let Some(batch) = input.next().await { @@ -1526,11 +1544,18 @@ impl Checksum { } } +#[derive(Debug, Clone)] +pub enum CompressionCodec { + None, + Zstd(i32), +} + /// Writes given record batch as Arrow IPC bytes into given writer. /// Returns number of bytes written. -pub(crate) fn write_ipc_compressed( +pub fn write_ipc_compressed( batch: &RecordBatch, output: &mut W, + codec: &CompressionCodec, ipc_time: &Time, ) -> Result { if batch.num_rows() == 0 { @@ -1543,14 +1568,24 @@ pub(crate) fn write_ipc_compressed( // write ipc_length placeholder output.write_all(&[0u8; 8])?; - // write ipc data - // TODO: make compression level configurable - let mut arrow_writer = StreamWriter::try_new(zstd::Encoder::new(output, 1)?, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; + let output = match codec { + CompressionCodec::None => { + let mut arrow_writer = StreamWriter::try_new(output, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + arrow_writer.into_inner()? + } + CompressionCodec::Zstd(level) => { + let encoder = zstd::Encoder::new(output, *level)?; + let mut arrow_writer = StreamWriter::try_new(encoder, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + let zstd_encoder = arrow_writer.into_inner()?; + zstd_encoder.finish()? + } + }; - let zwriter = arrow_writer.into_inner()?; - let output = zwriter.finish()?; + // fill ipc length let end_pos = output.stream_position()?; let ipc_length = end_pos - start_pos - 8; @@ -1611,6 +1646,22 @@ mod test { use datafusion_physical_expr::expressions::Column; use tokio::runtime::Runtime; + #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx` + fn write_ipc_zstd() { + let batch = create_batch(8192); + let mut output = vec![]; + let mut cursor = Cursor::new(&mut output); + write_ipc_compressed( + &batch, + &mut cursor, + &CompressionCodec::Zstd(1), + &Time::default(), + ) + .unwrap(); + assert_eq!(40218, output.len()); + } + #[test] fn test_slot_size() { let batch_size = 1usize; @@ -1673,13 +1724,7 @@ mod test { num_partitions: usize, memory_limit: Option, ) { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); - let mut b = StringBuilder::new(); - for i in 0..batch_size { - b.append_value(format!("{i}")); - } - let array = b.finish(); - let batch = RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap(); + let batch = create_batch(batch_size); let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); @@ -1687,6 +1732,7 @@ mod test { let exec = ShuffleWriterExec::try_new( Arc::new(MemoryExec::try_new(partitions, batch.schema(), None).unwrap()), Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + CompressionCodec::Zstd(1), "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), ) @@ -1707,6 +1753,16 @@ mod test { rt.block_on(collect(stream)).unwrap(); } + fn create_batch(batch_size: usize) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); + let mut b = StringBuilder::new(); + for i in 0..batch_size { + b.append_value(format!("{i}")); + } + let array = b.finish(); + RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() + } + #[test] fn test_pmod() { let i: Vec = vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb]; diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 74ec80cb54..5cb2802da8 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -82,10 +82,17 @@ message Limit { int32 offset = 2; } +enum CompressionCodec { + None = 0; + Zstd = 1; +} + message ShuffleWriter { spark.spark_partitioning.Partitioning partitioning = 1; string output_data_file = 3; string output_index_file = 4; + CompressionCodec codec = 5; + int32 compression_level = 6; } enum AggregateMode { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index 0cd8a9ce67..3a11b8b28c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -52,8 +52,9 @@ import org.apache.spark.util.random.XORShiftRandom import com.google.common.base.Objects +import org.apache.comet.CometConf import org.apache.comet.serde.{OperatorOuterClass, PartitioningOuterClass, QueryPlanSerde} -import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.OperatorOuterClass.{CompressionCodec, Operator} import org.apache.comet.serde.QueryPlanSerde.serializeDataType import org.apache.comet.shims.ShimCometShuffleExchangeExec @@ -553,6 +554,17 @@ class CometShuffleWriteProcessor( shuffleWriterBuilder.setOutputDataFile(dataFile) shuffleWriterBuilder.setOutputIndexFile(indexFile) + if (SparkEnv.get.conf.getBoolean("spark.shuffle.compress", true)) { + val codec = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get() match { + case "zstd" => CompressionCodec.Zstd + case other => throw new UnsupportedOperationException(s"invalid codec: $other") + } + shuffleWriterBuilder.setCodec(codec) + } else { + shuffleWriterBuilder.setCodec(CompressionCodec.None) + } + shuffleWriterBuilder.setCompressionLevel(CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_LEVEL.get) + outputPartitioning match { case _: HashPartitioning => val hashPartitioning = outputPartitioning.asInstanceOf[HashPartitioning] diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala index ef67167c4b..b2cc2c2bad 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala @@ -243,7 +243,7 @@ object CometShuffleManager extends Logging { lazy val compressionCodecForShuffling: CompressionCodec = { val sparkConf = SparkEnv.get.conf - val codecName = CometConf.COMET_EXEC_SHUFFLE_CODEC.get(SQLConf.get) + val codecName = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get(SQLConf.get) // only zstd compression is supported at the moment if (codecName != "zstd") { From 053b7ccc18c8397c7035a79b4df092beddf28289 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Dec 2024 17:59:34 -0700 Subject: [PATCH 41/47] minor: move shuffle classes from common to spark (#1193) --- .../execution/shuffle/ArrowReaderIterator.scala | 0 .../shuffle/CometBlockStoreShuffleReader.scala | 17 ++++------------- .../shuffle/CometShuffleDependency.scala | 0 .../execution/shuffle/CometShuffledRowRDD.scala | 4 ++-- .../shuffle/IpcInputStreamIterator.scala | 9 +++------ .../comet/execution/shuffle/ShuffleUtils.scala | 0 6 files changed, 9 insertions(+), 21 deletions(-) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala (100%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala (92%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala (100%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala (95%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala (95%) rename {common => spark}/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala (100%) diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala similarity index 92% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala index e026cbeb1a..74c6559504 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala @@ -21,21 +21,12 @@ package org.apache.spark.sql.comet.execution.shuffle import java.io.InputStream -import org.apache.spark.InterruptibleIterator -import org.apache.spark.MapOutputTracker -import org.apache.spark.SparkEnv -import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config +import org.apache.spark.{InterruptibleIterator, MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.SerializerManager -import org.apache.spark.shuffle.BaseShuffleHandle -import org.apache.spark.shuffle.ShuffleReader -import org.apache.spark.shuffle.ShuffleReadMetricsReporter -import org.apache.spark.storage.BlockId -import org.apache.spark.storage.BlockManager -import org.apache.spark.storage.BlockManagerId -import org.apache.spark.storage.ShuffleBlockFetcherIterator +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader, ShuffleReadMetricsReporter} +import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator /** diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala similarity index 95% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala index af78ed2905..ba6fc588e2 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.comet.execution.shuffle -import org.apache.spark.{Dependency, MapOutputTrackerMaster, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.execution.{CoalescedMapperPartitionSpec, CoalescedPartitioner, CoalescedPartitionSpec, PartialMapperPartitionSpec, PartialReducerPartitionSpec, ShufflePartitionSpec} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala similarity index 95% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala index d1d5af350f..aa40550488 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala @@ -19,12 +19,9 @@ package org.apache.spark.sql.comet.execution.shuffle -import java.io.EOFException -import java.io.InputStream -import java.nio.ByteBuffer -import java.nio.ByteOrder -import java.nio.channels.Channels -import java.nio.channels.ReadableByteChannel +import java.io.{EOFException, InputStream} +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.channels.{Channels, ReadableByteChannel} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala From 639fa2fb450a760728bc6921ca58da64df0c64a7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 22 Dec 2024 12:25:21 -0700 Subject: [PATCH 42/47] minor: refactor decodeBatches to make private in broadcast exchange (#1195) --- .../comet/CometBroadcastExchangeExec.scala | 24 +++++++++++-- .../apache/spark/sql/comet/operators.scala | 35 ++----------------- .../apache/comet/exec/CometExecSuite.scala | 33 ----------------- 3 files changed, 25 insertions(+), 67 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index ccf218cf6c..6bc519ab95 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.comet +import java.io.DataInputStream +import java.nio.channels.Channels import java.util.UUID import java.util.concurrent.{Future, TimeoutException, TimeUnit} @@ -26,13 +28,15 @@ import scala.concurrent.{ExecutionContext, Promise} import scala.concurrent.duration.NANOSECONDS import scala.util.control.NonFatal -import org.apache.spark.{broadcast, Partition, SparkContext, TaskContext} +import org.apache.spark.{broadcast, Partition, SparkContext, SparkEnv, TaskContext} import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec +import org.apache.spark.io.CompressionCodec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.comet.execution.shuffle.ArrowReaderIterator import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, ShuffleQueryStageExec} @@ -299,7 +303,23 @@ class CometBatchRDD( override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { val partition = split.asInstanceOf[CometBatchPartition] partition.value.value.toIterator - .flatMap(CometExec.decodeBatches(_, this.getClass.getSimpleName)) + .flatMap(decodeBatches(_, this.getClass.getSimpleName)) + } + + /** + * Decodes the byte arrays back to ColumnarBatchs and put them into buffer. + */ + private def decodeBatches(bytes: ChunkedByteBuffer, source: String): Iterator[ColumnarBatch] = { + if (bytes.size == 0) { + return Iterator.empty + } + + // use Spark's compression codec (LZ4 by default) and not Comet's compression + val codec = CompressionCodec.createCodec(SparkEnv.get.conf) + val cbbis = bytes.toInputStream() + val ins = new DataInputStream(codec.compressedInputStream(cbbis)) + // batches are in Arrow IPC format + new ArrowReaderIterator(Channels.newChannel(ins), source) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 77188312e4..c70f7464ec 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.comet -import java.io.{ByteArrayOutputStream, DataInputStream} -import java.nio.channels.Channels +import java.io.ByteArrayOutputStream import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{SparkEnv, TaskContext} -import org.apache.spark.io.CompressionCodec +import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, NamedExpression, SortOrder} @@ -34,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning} -import org.apache.spark.sql.comet.execution.shuffle.{ArrowReaderIterator, CometShuffleExchangeExec} +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} @@ -78,18 +76,6 @@ abstract class CometExec extends CometPlan { // outputPartitioning of SparkPlan, e.g., AQEShuffleReadExec. override def outputPartitioning: Partitioning = originalPlan.outputPartitioning - /** - * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. - */ - def executeColumnarCollectIterator(): (Long, Iterator[ColumnarBatch]) = { - val countsAndBytes = CometExec.getByteArrayRdd(this).collect() - val total = countsAndBytes.map(_._1).sum - val rows = countsAndBytes.iterator - .flatMap(countAndBytes => - CometExec.decodeBatches(countAndBytes._2, this.getClass.getSimpleName)) - (total, rows) - } - protected def setSubqueries(planId: Long, sparkPlan: SparkPlan): Unit = { sparkPlan.children.foreach(setSubqueries(planId, _)) @@ -161,21 +147,6 @@ object CometExec { Utils.serializeBatches(iter) } } - - /** - * Decodes the byte arrays back to ColumnarBatchs and put them into buffer. - */ - def decodeBatches(bytes: ChunkedByteBuffer, source: String): Iterator[ColumnarBatch] = { - if (bytes.size == 0) { - return Iterator.empty - } - - val codec = CompressionCodec.createCodec(SparkEnv.get.conf) - val cbbis = bytes.toInputStream() - val ins = new DataInputStream(codec.compressedInputStream(cbbis)) - - new ArrowReaderIterator(Channels.newChannel(ins), source) - } } /** diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 1027695372..90c3221e5d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -22,8 +22,6 @@ package org.apache.comet.exec import java.sql.Date import java.time.{Duration, Period} -import scala.collection.JavaConverters._ -import scala.collection.mutable import scala.util.Random import org.scalactic.source.Position @@ -462,37 +460,6 @@ class CometExecSuite extends CometTestBase { } } - test("CometExec.executeColumnarCollectIterator can collect ColumnarBatch results") { - assume(isSpark34Plus, "ChunkedByteBuffer is not serializable before Spark 3.4+") - withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "true") { - withParquetTable((0 until 50).map(i => (i, i + 1)), "tbl") { - val df = sql("SELECT _1 + 1, _2 + 2 FROM tbl WHERE _1 > 3") - - val nativeProject = find(df.queryExecution.executedPlan) { - case _: CometProjectExec => true - case _ => false - }.get.asInstanceOf[CometProjectExec] - - val (rows, batches) = nativeProject.executeColumnarCollectIterator() - assert(rows == 46) - - val column1 = mutable.ArrayBuffer.empty[Int] - val column2 = mutable.ArrayBuffer.empty[Int] - - batches.foreach(batch => { - batch.rowIterator().asScala.foreach { row => - assert(row.numFields == 2) - column1 += row.getInt(0) - column2 += row.getInt(1) - } - }) - - assert(column1.toArray.sorted === (4 until 50).map(_ + 1).toArray) - assert(column2.toArray.sorted === (5 until 51).map(_ + 2).toArray) - } - } - } - test("scalar subquery") { val dataTypes = Seq( From 58dee739b6b8c3e7c6057e01c72307cdcff56ada Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 22 Dec 2024 12:25:35 -0700 Subject: [PATCH 43/47] minor: refactor prepare_output so that it does not require an ExecutionContext (#1194) --- native/core/src/execution/jni_api.rs | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index eb73675b5b..2c1a55f48f 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -231,7 +231,7 @@ fn prepare_output( array_addrs: jlongArray, schema_addrs: jlongArray, output_batch: RecordBatch, - exec_context: &mut ExecutionContext, + validate: bool, ) -> CometResult { let array_address_array = unsafe { JLongArray::from_raw(array_addrs) }; let num_cols = env.get_array_length(&array_address_array)? as usize; @@ -255,7 +255,7 @@ fn prepare_output( ))); } - if exec_context.debug_native { + if validate { // Validate the output arrays. for array in results.iter() { let array_data = array.to_data(); @@ -275,9 +275,6 @@ fn prepare_output( i += 1; } - // Update metrics - update_metrics(env, exec_context)?; - Ok(num_rows as jlong) } @@ -356,22 +353,22 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( let next_item = exec_context.stream.as_mut().unwrap().next(); let poll_output = exec_context.runtime.block_on(async { poll!(next_item) }); + // Update metrics + update_metrics(&mut env, exec_context)?; + match poll_output { Poll::Ready(Some(output)) => { + // prepare output for FFI transfer return prepare_output( &mut env, array_addrs, schema_addrs, output?, - exec_context, + exec_context.debug_native, ); } Poll::Ready(None) => { // Reaches EOF of output. - - // Update metrics - update_metrics(&mut env, exec_context)?; - if exec_context.explain_native { if let Some(plan) = &exec_context.root_op { let formatted_plan_str = @@ -391,9 +388,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( // A poll pending means there are more than one blocking operators, // we don't need go back-forth between JVM/Native. Just keeping polling. Poll::Pending => { - // Update metrics - update_metrics(&mut env, exec_context)?; - // Pull input batches pull_input_batches(exec_context)?; From 5432e03e4c441257056638dca70f576e1e4c186b Mon Sep 17 00:00:00 2001 From: Raz Luvaton <16746759+rluvaton@users.noreply.github.com> Date: Fri, 27 Dec 2024 08:43:53 +0200 Subject: [PATCH 44/47] fix: fix missing explanation for then branch in case when (#1200) --- .../src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index b33f6b5a6b..518fa06858 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1745,7 +1745,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim exprToProtoInternal(elements._1, inputs) }) val thenSeq = branches.map(elements => { - allBranches = allBranches :+ elements._1 + allBranches = allBranches :+ elements._2 exprToProtoInternal(elements._2, inputs) }) assert(whenSeq.length == thenSeq.length) From 103f82f5491a48d8f0dbcb697741270d7fe268b8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Dec 2024 22:54:17 -0700 Subject: [PATCH 45/47] minor: remove unused source files (#1202) --- native/Cargo.lock | 48 +--- native/core/Cargo.toml | 4 - native/core/src/parquet/compression.rs | 319 --------------------- native/core/src/parquet/util/jni_buffer.rs | 98 ------- 4 files changed, 2 insertions(+), 467 deletions(-) delete mode 100644 native/core/src/parquet/compression.rs delete mode 100644 native/core/src/parquet/util/jni_buffer.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index 538c40ee23..ad572acb9e 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -428,17 +428,6 @@ dependencies = [ "generic-array", ] -[[package]] -name = "brotli" -version = "3.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", - "brotli-decompressor 2.5.1", -] - [[package]] name = "brotli" version = "7.0.0" @@ -447,17 +436,7 @@ checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", - "brotli-decompressor 4.0.1", -] - -[[package]] -name = "brotli-decompressor" -version = "2.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", + "brotli-decompressor", ] [[package]] @@ -900,7 +879,6 @@ dependencies = [ "arrow-schema", "assertables", "async-trait", - "brotli 3.5.0", "bytes", "crc32fast", "criterion", @@ -912,7 +890,6 @@ dependencies = [ "datafusion-expr", "datafusion-functions-nested", "datafusion-physical-expr", - "flate2", "futures", "hex", "itertools 0.11.0", @@ -920,7 +897,6 @@ dependencies = [ "lazy_static", "log", "log4rs", - "lz4", "mimalloc", "num", "once_cell", @@ -932,7 +908,6 @@ dependencies = [ "regex", "serde", "simd-adler32", - "snap", "tempfile", "thiserror", "tokio", @@ -2111,25 +2086,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "lz4" -version = "1.28.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d1febb2b4a79ddd1980eede06a8f7902197960aa0383ffcfdd62fe723036725" -dependencies = [ - "lz4-sys", -] - -[[package]] -name = "lz4-sys" -version = "1.11.1+lz4-1.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bd8c0d6c6ed0cd30b3652886bb8711dc4bb01d637a68105a3d5158039b418e6" -dependencies = [ - "cc", - "libc", -] - [[package]] name = "lz4_flex" version = "0.11.3" @@ -2382,7 +2338,7 @@ dependencies = [ "arrow-schema", "arrow-select", "base64", - "brotli 7.0.0", + "brotli", "bytes", "chrono", "flate2", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 489da46d47..5089e67a03 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -52,10 +52,6 @@ serde = { version = "1", features = ["derive"] } lazy_static = "1.4.0" prost = "0.12.1" jni = "0.21" -snap = "1.1" -brotli = "3.3" -flate2 = "1.0" -lz4 = "1.24" zstd = "0.11" rand = { workspace = true} num = { workspace = true } diff --git a/native/core/src/parquet/compression.rs b/native/core/src/parquet/compression.rs deleted file mode 100644 index 37b857f4a2..0000000000 --- a/native/core/src/parquet/compression.rs +++ /dev/null @@ -1,319 +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. - -//! Contains codec interface and supported codec implementations. -//! -//! See [`Compression`](crate::basic::Compression) enum for all available compression -//! algorithms. -//! -//! # Example -//! -//! ```no_run -//! use comet::parquet::{basic::Compression, compression::create_codec}; -//! -//! let mut codec = match create_codec(Compression::SNAPPY) { -//! Ok(Some(codec)) => codec, -//! _ => panic!(), -//! }; -//! -//! let data = vec![b'p', b'a', b'r', b'q', b'u', b'e', b't']; -//! let mut compressed = vec![]; -//! codec.compress(&data[..], &mut compressed).unwrap(); -//! -//! let mut output = vec![]; -//! codec.decompress(&compressed[..], &mut output).unwrap(); -//! -//! assert_eq!(output, data); -//! ``` - -use super::basic::Compression as CodecType; -use crate::errors::{ParquetError, ParquetResult as Result}; - -use brotli::Decompressor; -use flate2::{read, write, Compression}; -use snap::raw::{decompress_len, max_compress_len, Decoder, Encoder}; -use std::io::{copy, Read, Write}; - -/// Parquet compression codec interface. -#[allow(clippy::ptr_arg)] -pub trait Codec { - /// Compresses data stored in slice `input_buf` and writes the compressed result - /// to `output_buf`. - /// Note that you'll need to call `clear()` before reusing the same `output_buf` - /// across different `compress` calls. - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()>; - - /// Decompresses data stored in slice `input_buf` and writes output to `output_buf`. - /// Returns the total number of bytes written. - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result; -} - -/// Given the compression type `codec`, returns a codec used to compress and decompress -/// bytes for the compression type. -/// This returns `None` if the codec type is `UNCOMPRESSED`. -pub fn create_codec(codec: CodecType) -> Result>> { - match codec { - CodecType::BROTLI => Ok(Some(Box::new(BrotliCodec::new()))), - CodecType::GZIP => Ok(Some(Box::new(GZipCodec::new()))), - CodecType::SNAPPY => Ok(Some(Box::new(SnappyCodec::new()))), - CodecType::LZ4 => Ok(Some(Box::new(LZ4Codec::new()))), - CodecType::ZSTD => Ok(Some(Box::new(ZSTDCodec::new()))), - CodecType::UNCOMPRESSED => Ok(None), - _ => Err(nyi_err!("The codec type {} is not supported yet", codec)), - } -} - -/// Codec for Snappy compression format. -pub struct SnappyCodec { - decoder: Decoder, - encoder: Encoder, -} - -impl SnappyCodec { - /// Creates new Snappy compression codec. - pub(crate) fn new() -> Self { - Self { - decoder: Decoder::new(), - encoder: Encoder::new(), - } - } -} - -impl Codec for SnappyCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let len = decompress_len(input_buf)?; - output_buf.resize(len, 0); - self.decoder - .decompress(input_buf, output_buf) - .map_err(|e| e.into()) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let output_buf_len = output_buf.len(); - let required_len = max_compress_len(input_buf.len()); - output_buf.resize(output_buf_len + required_len, 0); - let n = self - .encoder - .compress(input_buf, &mut output_buf[output_buf_len..])?; - output_buf.truncate(output_buf_len + n); - Ok(()) - } -} - -/// Codec for GZIP compression algorithm. -pub struct GZipCodec {} - -impl GZipCodec { - /// Creates new GZIP compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -impl Codec for GZipCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let mut decoder = read::GzDecoder::new(input_buf); - decoder.read_to_end(output_buf).map_err(|e| e.into()) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = write::GzEncoder::new(output_buf, Compression::default()); - encoder.write_all(input_buf)?; - encoder.try_finish().map_err(|e| e.into()) - } -} - -const BROTLI_DEFAULT_BUFFER_SIZE: usize = 4096; -const BROTLI_DEFAULT_COMPRESSION_QUALITY: u32 = 1; // supported levels 0-9 -const BROTLI_DEFAULT_LG_WINDOW_SIZE: u32 = 22; // recommended between 20-22 - -/// Codec for Brotli compression algorithm. -pub struct BrotliCodec {} - -impl BrotliCodec { - /// Creates new Brotli compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -impl Codec for BrotliCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - Decompressor::new(input_buf, BROTLI_DEFAULT_BUFFER_SIZE) - .read_to_end(output_buf) - .map_err(|e| e.into()) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = brotli::CompressorWriter::new( - output_buf, - BROTLI_DEFAULT_BUFFER_SIZE, - BROTLI_DEFAULT_COMPRESSION_QUALITY, - BROTLI_DEFAULT_LG_WINDOW_SIZE, - ); - encoder.write_all(input_buf)?; - encoder.flush().map_err(|e| e.into()) - } -} - -const LZ4_BUFFER_SIZE: usize = 4096; - -/// Codec for LZ4 compression algorithm. -pub struct LZ4Codec {} - -impl LZ4Codec { - /// Creates new LZ4 compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -impl Codec for LZ4Codec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let mut decoder = lz4::Decoder::new(input_buf)?; - let mut buffer: [u8; LZ4_BUFFER_SIZE] = [0; LZ4_BUFFER_SIZE]; - let mut total_len = 0; - loop { - let len = decoder.read(&mut buffer)?; - if len == 0 { - break; - } - total_len += len; - output_buf.write_all(&buffer[0..len])?; - } - Ok(total_len) - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = lz4::EncoderBuilder::new().build(output_buf)?; - let mut from = 0; - loop { - let to = std::cmp::min(from + LZ4_BUFFER_SIZE, input_buf.len()); - encoder.write_all(&input_buf[from..to])?; - from += LZ4_BUFFER_SIZE; - if from >= input_buf.len() { - break; - } - } - encoder.finish().1.map_err(|e| e.into()) - } -} - -/// Codec for Zstandard compression algorithm. -pub struct ZSTDCodec {} - -impl ZSTDCodec { - /// Creates new Zstandard compression codec. - pub(crate) fn new() -> Self { - Self {} - } -} - -/// Compression level (1-21) for ZSTD. Choose 1 here for better compression speed. -const ZSTD_COMPRESSION_LEVEL: i32 = 1; - -impl Codec for ZSTDCodec { - fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result { - let mut decoder = zstd::Decoder::new(input_buf)?; - match copy(&mut decoder, output_buf) { - Ok(n) => Ok(n as usize), - Err(e) => Err(e.into()), - } - } - - fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec) -> Result<()> { - let mut encoder = zstd::Encoder::new(output_buf, ZSTD_COMPRESSION_LEVEL)?; - encoder.write_all(input_buf)?; - match encoder.finish() { - Ok(_) => Ok(()), - Err(e) => Err(e.into()), - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - - use crate::parquet::util::test_common::*; - - fn test_roundtrip(c: CodecType, data: &[u8]) { - let mut c1 = create_codec(c).unwrap().unwrap(); - let mut c2 = create_codec(c).unwrap().unwrap(); - - // Compress with c1 - let mut compressed = Vec::new(); - let mut decompressed = Vec::new(); - c1.compress(data, &mut compressed) - .expect("Error when compressing"); - - // Decompress with c2 - let mut decompressed_size = c2 - .decompress(compressed.as_slice(), &mut decompressed) - .expect("Error when decompressing"); - assert_eq!(data.len(), decompressed_size); - decompressed.truncate(decompressed_size); - assert_eq!(data, decompressed.as_slice()); - - compressed.clear(); - - // Compress with c2 - c2.compress(data, &mut compressed) - .expect("Error when compressing"); - - // Decompress with c1 - decompressed_size = c1 - .decompress(compressed.as_slice(), &mut decompressed) - .expect("Error when decompressing"); - assert_eq!(data.len(), decompressed_size); - decompressed.truncate(decompressed_size); - assert_eq!(data, decompressed.as_slice()); - } - - fn test_codec(c: CodecType) { - let sizes = vec![100, 10000, 100000]; - for size in sizes { - let data = random_bytes(size); - test_roundtrip(c, &data); - } - } - - #[test] - fn test_codec_snappy() { - test_codec(CodecType::SNAPPY); - } - - #[test] - fn test_codec_gzip() { - test_codec(CodecType::GZIP); - } - - #[test] - fn test_codec_brotli() { - test_codec(CodecType::BROTLI); - } - - #[test] - fn test_codec_lz4() { - test_codec(CodecType::LZ4); - } - - #[test] - fn test_codec_zstd() { - test_codec(CodecType::ZSTD); - } -} diff --git a/native/core/src/parquet/util/jni_buffer.rs b/native/core/src/parquet/util/jni_buffer.rs deleted file mode 100644 index 33f36ed9dd..0000000000 --- a/native/core/src/parquet/util/jni_buffer.rs +++ /dev/null @@ -1,98 +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. - -use core::slice; -use std::ptr::NonNull; - -use jni::{ - objects::{ReleaseMode, TypeArray}, - sys::{jbyte, jbyteArray, JNI_TRUE}, - JavaVM, -}; - -use crate::errors::{CometError, CometResult as Result}; - -use super::Buffer; - -/// An immutable byte buffer wrapping a JNI byte array allocated on heap. -/// -/// Unlike `AutoArray`, this doesn't have a lifetime and can be used across different JNI calls. -pub struct JniBuffer { - /// A pointer for the JVM instance, used to obtain byte array elements (via - /// `GetByteArrayElements`) and release byte array elements (via `ReleaseByteArrayElements`). - jvm: JavaVM, - /// The original JNI byte array that backs this buffer - inner: jbyteArray, - /// The raw pointer from the JNI byte array - ptr: NonNull, - /// Total number of bytes in the original array (i.e., `inner`). - len: usize, - /// Whether the JNI byte array is copied or not. - is_copy: bool, -} - -impl JniBuffer { - pub fn try_new(jvm: JavaVM, array: jbyteArray, len: usize) -> Result { - let env = jvm.get_env()?; - let mut is_copy = 0xff; - let ptr = jbyte::get(&env, array.into(), &mut is_copy)?; - let res = Self { - jvm, - inner: array, - ptr: NonNull::new(ptr) - .ok_or_else(|| CometError::NullPointer("null byte array pointer".to_string()))?, - len, - is_copy: is_copy == JNI_TRUE, - }; - Ok(res) - } - - /// Whether the JNI byte array is copied or not, i.e., whether the JVM pinned down the original - /// Java byte array, or made a new copy of it. - pub fn is_copy(&self) -> bool { - self.is_copy - } -} - -impl Buffer for JniBuffer { - fn len(&self) -> usize { - self.len - } - - fn data(&self) -> &[u8] { - self.as_ref() - } -} - -impl AsRef<[u8]> for JniBuffer { - fn as_ref(&self) -> &[u8] { - unsafe { slice::from_raw_parts(self.ptr.as_ptr() as *mut u8 as *const u8, self.len) } - } -} - -impl Drop for JniBuffer { - fn drop(&mut self) { - let env = self.jvm.get_env().unwrap(); // TODO: log error here - jbyte::release( - &env, - self.inner.into(), - self.ptr, - ReleaseMode::NoCopyBack as i32, // don't copy back since it's read-only here - ) - .unwrap(); - } -} From 5d2c909f59caf90348096511d4a26e48a0ae8e3d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Dec 2024 15:14:11 -0700 Subject: [PATCH 46/47] chore: Upgrade to DataFusion 44.0.0-rc2 (#1154) * move aggregate expressions to spark-expr crate * move more expressions * move benchmark * normalize_nan * bitwise not * comet scalar funcs * update bench imports * save * save * save * remove unused imports * clippy * implement more hashers * implement Hash and PartialEq * implement Hash and PartialEq * implement Hash and PartialEq * benches * fix ScalarUDFImpl.return_type failure * exclude test from miri * ignore correct test * ignore another test * remove miri checks * use return_type_from_exprs * Revert "use return_type_from_exprs" This reverts commit febc1f1ec1301f9b359fc23ad6a117224fce35b7. * use DF main branch * hacky workaround for regression in ScalarUDFImpl.return_type * fix repo url * pin to revision * bump to latest rev * bump to latest DF rev * bump DF to rev 9f530dd * add Cargo.lock * bump DF version * no default features * Revert "remove miri checks" This reverts commit 4638fe3aa5501966cd5d8b53acf26c698b10b3c9. * Update pin to DataFusion e99e02b9b9093ceb0c13a2dd32a2a89beba47930 * update pin * Update Cargo.toml Bump to 44.0.0-rc2 * update cargo lock * revert miri change --------- Co-authored-by: Andrew Lamb --- native/Cargo.lock | 598 +++++++++--------- native/Cargo.toml | 29 +- .../expressions/bloom_filter_might_contain.rs | 46 +- .../src/execution/expressions/subquery.rs | 23 +- native/core/src/execution/jni_api.rs | 9 +- native/core/src/execution/operators/copy.rs | 4 +- native/core/src/execution/operators/expand.rs | 6 +- native/core/src/execution/operators/filter.rs | 3 +- native/core/src/execution/operators/scan.rs | 4 +- native/core/src/execution/planner.rs | 55 +- .../src/execution/shuffle/shuffle_writer.rs | 6 +- .../src/execution/util/spark_bit_array.rs | 2 +- .../src/execution/util/spark_bloom_filter.rs | 2 +- native/spark-expr/Cargo.toml | 1 + native/spark-expr/benches/aggregate.rs | 3 +- native/spark-expr/src/avg.rs | 22 +- native/spark-expr/src/avg_decimal.rs | 20 +- native/spark-expr/src/bitwise_not.rs | 38 +- native/spark-expr/src/cast.rs | 42 +- native/spark-expr/src/checkoverflow.rs | 53 +- native/spark-expr/src/correlation.rs | 29 +- native/spark-expr/src/covariance.rs | 26 +- native/spark-expr/src/if_expr.rs | 49 +- native/spark-expr/src/list.rs | 127 ++-- native/spark-expr/src/negative.rs | 37 +- native/spark-expr/src/normalize_nan.rs | 45 +- native/spark-expr/src/regexp.rs | 27 +- native/spark-expr/src/stddev.rs | 30 +- native/spark-expr/src/strings.rs | 89 ++- native/spark-expr/src/structs.rs | 59 +- native/spark-expr/src/sum_decimal.rs | 28 +- native/spark-expr/src/temporal.rs | 175 +++-- native/spark-expr/src/to_json.rs | 23 +- native/spark-expr/src/unbound.rs | 21 +- native/spark-expr/src/variance.rs | 19 +- 35 files changed, 715 insertions(+), 1035 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index ad572acb9e..bbc0ff97a9 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1,6 +1,6 @@ # This file is automatically @generated by Cargo. # It is not intended for manual editing. -version = 3 +version = 4 [[package]] name = "addr2line" @@ -57,9 +57,9 @@ dependencies = [ [[package]] name = "allocator-api2" -version = "0.2.20" +version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45862d1c77f2228b9e10bc609d5bc203d86ebc9b87ad8d5d5167a6c9abf739d9" +checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" [[package]] name = "android-tzdata" @@ -90,9 +90,9 @@ checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anyhow" -version = "1.0.93" +version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c95c10ba0b00a02636238b814946408b1322d5ac4760326e6fb8ec956d85775" +checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" [[package]] name = "arc-swap" @@ -114,9 +114,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4caf25cdc4a985f91df42ed9e9308e1adbcd341a31a72605c697033fcef163e3" +checksum = "c91839b07e474b3995035fd8ac33ee54f9c9ccbbb1ea33d9909c71bffdf1259d" dependencies = [ "arrow-arith", "arrow-array", @@ -135,9 +135,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91f2dfd1a7ec0aca967dfaa616096aec49779adc8eccec005e2f5e4111b1192a" +checksum = "855c57c4efd26722b044dcd3e348252560e3e0333087fb9f6479dc0bf744054f" dependencies = [ "arrow-array", "arrow-buffer", @@ -150,9 +150,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d39387ca628be747394890a6e47f138ceac1aa912eab64f02519fed24b637af8" +checksum = "bd03279cea46569acf9295f6224fbc370c5df184b4d2ecfe97ccb131d5615a7f" dependencies = [ "ahash", "arrow-buffer", @@ -161,15 +161,15 @@ dependencies = [ "chrono", "chrono-tz 0.10.0", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.2", "num", ] [[package]] name = "arrow-buffer" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e51e05228852ffe3eb391ce7178a0f97d2cf80cc6ef91d3c4a6b3cb688049ec" +checksum = "9e4a9b9b1d6d7117f6138e13bc4dd5daa7f94e671b70e8c9c4dc37b4f5ecfc16" dependencies = [ "bytes", "half", @@ -178,9 +178,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d09aea56ec9fa267f3f3f6cdab67d8a9974cbba90b3aa38c8fe9d0bb071bd8c1" +checksum = "bc70e39916e60c5b7af7a8e2719e3ae589326039e1e863675a008bee5ffe90fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -199,9 +199,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c07b5232be87d115fde73e32f2ca7f1b353bff1b44ac422d3c6fc6ae38f11f0d" +checksum = "789b2af43c1049b03a8d088ff6b2257cdcea1756cd76b174b1f2600356771b97" dependencies = [ "arrow-array", "arrow-buffer", @@ -218,9 +218,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b98ae0af50890b494cebd7d6b04b35e896205c1d1df7b29a6272c5d0d0249ef5" +checksum = "e4e75edf21ffd53744a9b8e3ed11101f610e7ceb1a29860432824f1834a1f623" dependencies = [ "arrow-buffer", "arrow-schema", @@ -230,9 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ed91bdeaff5a1c00d28d8f73466bcb64d32bbd7093b5a30156b4b9f4dba3eee" +checksum = "d186a909dece9160bf8312f5124d797884f608ef5435a36d9d608e0b2a9bcbf8" dependencies = [ "arrow-array", "arrow-buffer", @@ -245,9 +245,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0471f51260a5309307e5d409c9dc70aede1cd9cf1d4ff0f0a1e8e1a2dd0e0d3c" +checksum = "b66ff2fedc1222942d0bd2fd391cb14a85baa3857be95c9373179bd616753b85" dependencies = [ "arrow-array", "arrow-buffer", @@ -265,9 +265,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2883d7035e0b600fb4c30ce1e50e66e53d8656aa729f2bfa4b51d359cf3ded52" +checksum = "ece7b5bc1180e6d82d1a60e1688c199829e8842e38497563c3ab6ea813e527fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,9 +280,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "552907e8e587a6fde4f8843fd7a27a576a260f65dab6c065741ea79f633fc5be" +checksum = "745c114c8f0e8ce211c83389270de6fbe96a9088a7b32c2a041258a443fe83ff" dependencies = [ "ahash", "arrow-array", @@ -294,18 +294,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "539ada65246b949bd99ffa0881a9a15a4a529448af1a07a9838dd78617dafab1" +checksum = "b95513080e728e4cec37f1ff5af4f12c9688d47795d17cda80b6ec2cf74d4678" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "arrow-select" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6259e566b752da6dceab91766ed8b2e67bf6270eb9ad8a6e07a33c1bede2b125" +checksum = "8e415279094ea70323c032c6e739c48ad8d80e78a09bef7117b8718ad5bf3722" dependencies = [ "ahash", "arrow-array", @@ -317,9 +317,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3179ccbd18ebf04277a095ba7321b93fd1f774f18816bd5f6b3ce2f594edb6c" +checksum = "11d956cae7002eb8d83a27dbd34daaea1cf5b75852f0b84deb4d93a276e92bbf" dependencies = [ "arrow-array", "arrow-buffer", @@ -346,7 +346,7 @@ checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -385,6 +385,19 @@ version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" +[[package]] +name = "bigdecimal" +version = "0.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f31f3af01c5c65a07985c804d3366560e6fa7883d640a122819b14ec327482c" +dependencies = [ + "autocfg", + "libm", + "num-bigint", + "num-integer", + "num-traits", +] + [[package]] name = "bitflags" version = "1.3.2" @@ -408,9 +421,9 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.4" +version = "1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d82033247fd8e890df8f740e407ad4d038debb9eb1f40533fffb32e7d17dc6f7" +checksum = "b8ee0c1824c4dea5b5f81736aff91bae041d2c07ee1192bec91054e10e3e601e" dependencies = [ "arrayref", "arrayvec", @@ -457,9 +470,9 @@ checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytemuck" -version = "1.19.0" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8334215b81e418a0a7bdb8ef0849474f40bb10c8b71f1c4ed315cff49f32494d" +checksum = "ef657dfab802224e671f5818e9a4935f9b1957ed18e58292690cc39e7a4092a3" [[package]] name = "byteorder" @@ -469,9 +482,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.8.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ac0150caa2ae65ca5bd83f25c7de183dea78d4d366469f148435e2acfbad0da" +checksum = "325918d6fe32f23b19878fe4b34794ae41fc19ddbe53b10571a4874d44ffd39b" [[package]] name = "cast" @@ -481,9 +494,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.1" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd9de9f2205d5ef3fd67e685b0df337994ddd4495e2a28d185500d0e1edfea47" +checksum = "8d6dbb628b8f8555f86d0323c2eb39e3ec81901f4b83e091db8a6a76d316a333" dependencies = [ "jobserver", "libc", @@ -504,9 +517,9 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.38" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" +checksum = "7e36cc9d416881d2e24f9a963be5fb1cd90966419ac844274161d10488b3e825" dependencies = [ "android-tzdata", "iana-time-zone", @@ -586,18 +599,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.21" +version = "4.5.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb3b4b9e5a7c7514dfa52869339ee98b3156b0bfb4e8a77c4ff4babb64b1604f" +checksum = "3135e7ec2ef7b10c6ed8950f0f792ed96ee093fa088608f1c76e569722700c84" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.21" +version = "4.5.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b17a95aa67cc7b5ebd32aa5370189aa0d79069ef1c64ce893bd30fb24bff20ec" +checksum = "30582fc632330df2bd26877bde0c1f4470d57c582bbc070376afcd04d8cb4838" dependencies = [ "anstyle", "clap_lex", @@ -605,9 +618,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "afb84c814227b90d6895e01398aee0d8033c00e7466aca416fb6a8e0eb19d8a7" +checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" [[package]] name = "combine" @@ -673,9 +686,9 @@ dependencies = [ [[package]] name = "cpufeatures" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ca741a962e1b0bff6d724a1a0958b686406e853bb14061f218562e1896f95e6" +checksum = "16b80225097f2e5ae4e7179dd2266824648f3e2f49d9134d584b76389d31c4c3" dependencies = [ "libc", ] @@ -729,9 +742,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.5" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" dependencies = [ "crossbeam-epoch", "crossbeam-utils", @@ -748,9 +761,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.20" +version = "0.8.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" [[package]] name = "crunchy" @@ -805,11 +818,9 @@ dependencies = [ [[package]] name = "datafusion" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbba0799cf6913b456ed07a94f0f3b6e12c62a5d88b10809e2284a0f2b915c05" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ - "ahash", "arrow", "arrow-array", "arrow-ipc", @@ -825,6 +836,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", + "datafusion-functions-table", "datafusion-functions-window", "datafusion-optimizer", "datafusion-physical-expr", @@ -834,18 +846,13 @@ dependencies = [ "datafusion-sql", "futures", "glob", - "half", - "hashbrown 0.14.5", - "indexmap", "itertools 0.13.0", "log", - "num_cpus", "object_store", "parking_lot", "parquet", - "paste", - "pin-project-lite", "rand", + "regex", "sqlparser", "tempfile", "tokio", @@ -855,9 +862,8 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7493c5c2d40eec435b13d92e5703554f4efc7059451fcb8d3a79580ff0e45560" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow-schema", "async-trait", @@ -937,6 +943,7 @@ dependencies = [ "datafusion", "datafusion-common", "datafusion-expr", + "datafusion-expr-common", "datafusion-physical-expr", "futures", "num", @@ -945,57 +952,56 @@ dependencies = [ "regex", "thiserror", "tokio", - "twox-hash 2.0.1", + "twox-hash 2.1.0", ] [[package]] name = "datafusion-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24953049ebbd6f8964f91f60aa3514e121b5e81e068e33b60e77815ab369b25c" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "arrow-array", "arrow-buffer", "arrow-schema", - "chrono", "half", "hashbrown 0.14.5", "indexmap", - "instant", "libc", - "num_cpus", + "log", "object_store", "parquet", "paste", "sqlparser", "tokio", + "web-time", ] [[package]] name = "datafusion-common-runtime" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f06df4ef76872e11c924d3c814fd2a8dd09905ed2e2195f71c857d78abd19685" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "log", "tokio", ] +[[package]] +name = "datafusion-doc" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" + [[package]] name = "datafusion-execution" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bbdcb628d690f3ce5fea7de81642b514486d58ff9779a51f180a69a4eadb361" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", - "chrono", "dashmap", "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.5", "log", "object_store", "parking_lot", @@ -1006,16 +1012,13 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8036495980e3131f706b7d33ab00b4492d73dc714e3cb74d11b50f9602a73246" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ - "ahash", "arrow", - "arrow-array", - "arrow-buffer", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-expr-common", "datafusion-functions-aggregate-common", "datafusion-functions-window-common", @@ -1024,27 +1027,22 @@ dependencies = [ "paste", "serde_json", "sqlparser", - "strum", - "strum_macros", ] [[package]] name = "datafusion-expr-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4da0f3cb4669f9523b403d6b5a0ec85023e0ab3bf0183afd1517475b3e64fdd2" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "datafusion-common", "itertools 0.13.0", - "paste", ] [[package]] name = "datafusion-functions" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f52c4012648b34853e40a2c6bcaa8772f837831019b68aca384fb38436dba162" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-buffer", @@ -1053,8 +1051,11 @@ dependencies = [ "blake3", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-execution", "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", "hashbrown 0.14.5", "hex", "itertools 0.13.0", @@ -1069,44 +1070,41 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5b8bb624597ba28ed7446df4a9bd7c7a7bde7c578b6b527da3f47371d5f6741" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "arrow-schema", "datafusion-common", + "datafusion-doc", "datafusion-execution", "datafusion-expr", "datafusion-functions-aggregate-common", + "datafusion-macros", "datafusion-physical-expr", "datafusion-physical-expr-common", "half", - "indexmap", "log", "paste", ] [[package]] name = "datafusion-functions-aggregate-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fb06208fc470bc8cf1ce2d9a1159d42db591f2c7264a8c1776b53ad8f675143" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "datafusion-common", "datafusion-expr-common", "datafusion-physical-expr-common", - "rand", ] [[package]] name = "datafusion-functions-nested" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fca25bbb87323716d05e54114666e942172ccca23c5a507e9c7851db6e965317" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-array", @@ -1122,18 +1120,33 @@ dependencies = [ "itertools 0.13.0", "log", "paste", - "rand", +] + +[[package]] +name = "datafusion-functions-table" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" +dependencies = [ + "arrow", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-plan", + "parking_lot", + "paste", ] [[package]] name = "datafusion-functions-window" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ae23356c634e54c59f7c51acb7a5b9f6240ffb2cf997049a1a24a8a88598dbe" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "datafusion-common", + "datafusion-doc", "datafusion-expr", "datafusion-functions-window-common", + "datafusion-macros", "datafusion-physical-expr", "datafusion-physical-expr-common", "log", @@ -1142,48 +1155,49 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4b3d6ff7794acea026de36007077a06b18b89e4f9c3fea7f2215f9f7dd9059b" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", ] +[[package]] +name = "datafusion-macros" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" +dependencies = [ + "quote", + "syn 2.0.92", +] + [[package]] name = "datafusion-optimizer" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec6241eb80c595fa0e1a8a6b69686b5cf3bd5fdacb8319582a0943b0bd788aa" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", - "async-trait", "chrono", "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.5", "indexmap", "itertools 0.13.0", "log", - "paste", + "regex", "regex-syntax", ] [[package]] name = "datafusion-physical-expr" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3370357b8fc75ec38577700644e5d1b0bc78f38babab99c0b8bd26bafb3e4335" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "arrow-array", "arrow-buffer", - "arrow-ord", "arrow-schema", - "arrow-string", - "chrono", "datafusion-common", "datafusion-expr", "datafusion-expr-common", @@ -1200,39 +1214,36 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8b7734d94bf2fa6f6e570935b0ddddd8421179ce200065be97874e13d46a47b" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", "datafusion-common", "datafusion-expr-common", "hashbrown 0.14.5", - "rand", + "itertools 0.13.0", ] [[package]] name = "datafusion-physical-optimizer" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eee8c479522df21d7b395640dff88c5ed05361852dce6544d7c98e9dbcebffe" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", - "arrow-schema", "datafusion-common", "datafusion-execution", "datafusion-expr-common", "datafusion-physical-expr", "datafusion-physical-plan", "itertools 0.13.0", + "log", ] [[package]] name = "datafusion-physical-plan" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17e1fc2e2c239d14e8556f2622b19a726bf6bc6962cc00c71fc52626274bee24" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -1246,7 +1257,6 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", @@ -1256,29 +1266,26 @@ dependencies = [ "indexmap", "itertools 0.13.0", "log", - "once_cell", "parking_lot", "pin-project-lite", - "rand", "tokio", ] [[package]] name = "datafusion-sql" -version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63e3a4ed41dbee20a5d947a59ca035c225d67dc9cbe869c10f66dcdf25e7ce51" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-array", "arrow-schema", + "bigdecimal", "datafusion-common", "datafusion-expr", "indexmap", "log", "regex", "sqlparser", - "strum", ] [[package]] @@ -1326,7 +1333,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -1343,19 +1350,19 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.9" +version = "0.3.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" +checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" dependencies = [ "libc", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "fastrand" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "486f806e73c5707928240ddc295403b1b93c96a02038563881c4a2fd84b81ac4" +checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "findshlibs" @@ -1377,9 +1384,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flatbuffers" -version = "24.3.25" +version = "24.12.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" +checksum = "4f1baf0dbf96932ec9a3038d57900329c015b0bfb7b63d904f3bc27e2b02a096" dependencies = [ "bitflags 1.3.2", "rustc_version", @@ -1466,7 +1473,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -1555,9 +1562,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.15.1" +version = "0.15.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a9bfc1af68b1726ea47d3d5109de126281def866b33970e10fbab11b5dafab3" +checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" [[package]] name = "heck" @@ -1574,12 +1581,6 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" -[[package]] -name = "hermit-abi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" - [[package]] name = "hermit-abi" version = "0.4.0" @@ -1594,11 +1595,11 @@ checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" [[package]] name = "home" -version = "0.5.9" +version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +checksum = "589533453244b0995c858700322199b2becb13b627df2851f64a2775d024abcf" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -1745,7 +1746,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -1771,12 +1772,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.6.0" +version = "2.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" +checksum = "62f822373a4fe84d4bb149bf54e584a7f4abec90e072ed49cda0edea5b95471f" dependencies = [ "equivalent", - "hashbrown 0.15.1", + "hashbrown 0.15.2", ] [[package]] @@ -1797,18 +1798,6 @@ dependencies = [ "str_stack", ] -[[package]] -name = "instant" -version = "0.1.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" -dependencies = [ - "cfg-if", - "js-sys", - "wasm-bindgen", - "web-sys", -] - [[package]] name = "integer-encoding" version = "3.0.4" @@ -1821,7 +1810,7 @@ version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "261f68e344040fbd0edea105bef17c66edf46f984ddb1115b775ce31be948f4b" dependencies = [ - "hermit-abi 0.4.0", + "hermit-abi", "libc", "windows-sys 0.52.0", ] @@ -1864,9 +1853,9 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.11" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" +checksum = "d75a2a4b1b190afb6f5425f10f6a8f959d2ea0b9c2b1d79553551850539e4674" [[package]] name = "java-locator" @@ -1912,10 +1901,11 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.72" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a88f1bda2bd75b0452a14784937d796722fdebfe50df998aeb3f0b7603019a9" +checksum = "6717b6b5b077764fb5966237269cb3c64edddde4b14ce42647430a78ced9e7b7" dependencies = [ + "once_cell", "wasm-bindgen", ] @@ -1927,9 +1917,9 @@ checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" [[package]] name = "lexical-core" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0431c65b318a590c1de6b8fd6e72798c92291d27762d94c9e6c37ed7a73d8458" +checksum = "b765c31809609075565a70b4b71402281283aeda7ecaf4818ac14a7b2ade8958" dependencies = [ "lexical-parse-float", "lexical-parse-integer", @@ -1940,9 +1930,9 @@ dependencies = [ [[package]] name = "lexical-parse-float" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb17a4bdb9b418051aa59d41d65b1c9be5affab314a872e5ad7f06231fb3b4e0" +checksum = "de6f9cb01fb0b08060209a057c048fcbab8717b4c1ecd2eac66ebfe39a65b0f2" dependencies = [ "lexical-parse-integer", "lexical-util", @@ -1951,9 +1941,9 @@ dependencies = [ [[package]] name = "lexical-parse-integer" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5df98f4a4ab53bf8b175b363a34c7af608fe31f93cc1fb1bf07130622ca4ef61" +checksum = "72207aae22fc0a121ba7b6d479e42cbfea549af1479c3f3a4f12c70dd66df12e" dependencies = [ "lexical-util", "static_assertions", @@ -1961,18 +1951,18 @@ dependencies = [ [[package]] name = "lexical-util" -version = "1.0.3" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85314db53332e5c192b6bca611fb10c114a80d1b831ddac0af1e9be1b9232ca0" +checksum = "5a82e24bf537fd24c177ffbbdc6ebcc8d54732c35b50a3f28cc3f4e4c949a0b3" dependencies = [ "static_assertions", ] [[package]] name = "lexical-write-float" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e7c3ad4e37db81c1cbe7cf34610340adc09c322871972f74877a712abc6c809" +checksum = "c5afc668a27f460fb45a81a757b6bf2f43c2d7e30cb5a2dcd3abf294c78d62bd" dependencies = [ "lexical-util", "lexical-write-integer", @@ -1981,9 +1971,9 @@ dependencies = [ [[package]] name = "lexical-write-integer" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb89e9f6958b83258afa3deed90b5de9ef68eef090ad5086c791cd2345610162" +checksum = "629ddff1a914a836fb245616a7888b62903aae58fa771e1d83943035efa0f978" dependencies = [ "lexical-util", "static_assertions", @@ -1991,9 +1981,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.162" +version = "0.2.169" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18d287de67fe55fd7e1581fe933d965a5a9477b38e949cfa9f8574ef01506398" +checksum = "b5aba8db14291edd000dfcc4d620c7ebfb122c613afb886ca8803fa4e128a20a" [[package]] name = "libloading" @@ -2029,9 +2019,9 @@ checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" [[package]] name = "litemap" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "643cb0b8d4fcc284004d5fd0d67ccf61dfffadb7f75e1e71bc420f4688a3a704" +checksum = "4ee93343901ab17bd981295f2cf0026d4ad018c7c31ba84549a4ddbb47a45104" [[package]] name = "lock_api" @@ -2131,9 +2121,9 @@ dependencies = [ [[package]] name = "miniz_oxide" -version = "0.8.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" +checksum = "4ffbe83022cedc1d264172192511ae958937694cd57ce297164951b8b3568394" dependencies = [ "adler2", ] @@ -2239,30 +2229,20 @@ dependencies = [ "libm", ] -[[package]] -name = "num_cpus" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" -dependencies = [ - "hermit-abi 0.3.9", - "libc", -] - [[package]] name = "object" -version = "0.36.5" +version = "0.36.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" +checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eb4c22c6154a1e759d7099f9ffad7cc5ef8245f9efbab4a41b92623079c82f3" +checksum = "3cfccb68961a56facde1163f9319e0d15743352344e7808a11795fb99698dcaf" dependencies = [ "async-trait", "bytes", @@ -2325,9 +2305,9 @@ dependencies = [ [[package]] name = "parquet" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dea02606ba6f5e856561d8d507dba8bac060aefca2a6c0f1aa1d361fed91ff3e" +checksum = "2b449890367085eb65d7d3321540abc3d7babbd179ce31df0016e90719114191" dependencies = [ "ahash", "arrow-array", @@ -2344,7 +2324,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.2", "lz4_flex", "num", "num-bigint", @@ -2506,9 +2486,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.89" +version = "1.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f139b0662de085916d1fb67d2b4169d1addddda1919e696f3252b740b629986e" +checksum = "37d3544b3f2748c54e147655edb5025752e2303145b5aefb3c3ea2c78b973bb0" dependencies = [ "unicode-ident", ] @@ -2576,7 +2556,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2600,9 +2580,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.37" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" +checksum = "0e4dccaaaf89514f546c693ddc140f729f958c247918a13380cccc6078391acc" dependencies = [ "proc-macro2", ] @@ -2659,9 +2639,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.7" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" +checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" dependencies = [ "bitflags 2.6.0", ] @@ -2721,22 +2701,22 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.40" +version = "0.38.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99e4ea3e1cdc4b559b8e5650f9c8e5998e3e5c1343b4eaf034565f32318d63c0" +checksum = "f93dc38ecbab2eb790ff964bb77fa94faf256fd3e73285fd7ba0903b76bedb85" dependencies = [ "bitflags 2.6.0", "errno", "libc", "linux-raw-sys", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "rustversion" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" +checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "ryu" @@ -2761,9 +2741,9 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "semver" -version = "1.0.23" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" +checksum = "3cb6eb87a131f756572d7fb904f6e7b68633f09cca868c5df1c4b8d1a694bbba" [[package]] name = "seq-macro" @@ -2773,9 +2753,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.215" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6513c1ad0b11a9376da888e3e0baa0077f1aed55c17f50e7b2397136129fb88f" +checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" dependencies = [ "serde_derive", ] @@ -2792,20 +2772,20 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.215" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad1e866f866923f252f05c889987993144fb74e722403468a4ebd70c3cd756c0" +checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "serde_json" -version = "1.0.132" +version = "1.0.134" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d726bfaff4b320266d395898905d0eba0345aae23b54aee3a737e260fd46db03" +checksum = "d00f4175c42ee48b15416f6193a959ba3a0d67fc699a0db9ad12df9f83991c7d" dependencies = [ "itoa", "memchr", @@ -2888,7 +2868,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2899,9 +2879,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "sqlparser" -version = "0.51.0" +version = "0.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fe11944a61da0da3f592e19a45ebe5ab92dc14a779907ff1f08fbb797bfefc7" +checksum = "05a528114c392209b3264855ad491fcce534b94a38771b0a0b97a79379275ce8" dependencies = [ "log", "sqlparser_derive", @@ -2909,13 +2889,13 @@ dependencies = [ [[package]] name = "sqlparser_derive" -version = "0.2.2" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" +checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2941,9 +2921,6 @@ name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" -dependencies = [ - "strum_macros", -] [[package]] name = "strum_macros" @@ -2955,7 +2932,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -2966,9 +2943,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbolic-common" -version = "12.12.1" +version = "12.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d4d73159efebfb389d819fd479afb2dbd57dcb3e3f4b7fcfa0e675f5a46c1cb" +checksum = "cd33e73f154e36ec223c18013f7064a2c120f1162fc086ac9933542def186b00" dependencies = [ "debugid", "memmap2", @@ -2978,9 +2955,9 @@ dependencies = [ [[package]] name = "symbolic-demangle" -version = "12.12.1" +version = "12.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a767859f6549c665011970874c3f541838b4835d5aaaa493d3ee383918be9f10" +checksum = "89e51191290147f071777e37fe111800bb82a9059f9c95b19d2dd41bfeddf477" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -3000,9 +2977,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.87" +version = "2.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25aa4ce346d03a6dcd68dd8b4010bcb74e54e62c90c573f394c46eae99aba32d" +checksum = "70ae51629bf965c5c098cc9e87908a3df5301051a9e087d6f9bef5c9771ed126" dependencies = [ "proc-macro2", "quote", @@ -3017,7 +2994,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -3050,7 +3027,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] @@ -3105,9 +3082,9 @@ dependencies = [ [[package]] name = "tokio" -version = "1.41.1" +version = "1.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22cfb5bee7a6a52939ca9224d6ac897bb669134078daa8735560897f69de4d33" +checksum = "5cec9b21b0450273377fc97bd4c33a8acffc8c996c987a7c5b319a0083707551" dependencies = [ "backtrace", "bytes", @@ -3123,14 +3100,14 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "tracing" -version = "0.1.40" +version = "0.1.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" +checksum = "784e0ac535deb450455cbfa28a6f0df145ea1bb7ae51b821cf5e7927fdcfbdd0" dependencies = [ "pin-project-lite", "tracing-attributes", @@ -3139,20 +3116,20 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" +checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "tracing-core" -version = "0.1.32" +version = "0.1.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" +checksum = "e672c95779cf947c5311f83787af4fa8fffd12fb27e4993211a84bdfd9610f9c" dependencies = [ "once_cell", ] @@ -3169,9 +3146,9 @@ dependencies = [ [[package]] name = "twox-hash" -version = "2.0.1" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6db6856664807f43c17fbaf2718e2381ac1476a449aa104f5f64622defa1245" +checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" dependencies = [ "rand", ] @@ -3193,9 +3170,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-ident" -version = "1.0.13" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e91b56cd4cadaeb79bbf1a5645f6b4f8dc5bde8834ad5894a8db35fda9efa1fe" +checksum = "adb9e6ca4f869e1180728b7950e35922a7fc6397f7b641499e8f3ef06e50dc83" [[package]] name = "unicode-segmentation" @@ -3226,9 +3203,9 @@ checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" [[package]] name = "url" -version = "2.5.3" +version = "2.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d157f1b96d14500ffdc1f10ba712e780825526c03d9a49b4d0324b0d9113ada" +checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" dependencies = [ "form_urlencoded", "idna", @@ -3280,9 +3257,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "128d1e363af62632b8eb57219c8fd7877144af57558fb2ef0368d0087bddeb2e" +checksum = "a474f6281d1d70c17ae7aa6a613c87fce69a127e2624002df63dcb39d6cf6396" dependencies = [ "cfg-if", "once_cell", @@ -3291,24 +3268,23 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb6dd4d3ca0ddffd1dd1c9c04f94b868c37ff5fac97c30b97cff2d74fce3a358" +checksum = "5f89bb38646b4f81674e8f5c3fb81b562be1fd936d84320f3264486418519c79" dependencies = [ "bumpalo", "log", - "once_cell", "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e79384be7f8f5a9dd5d7167216f022090cf1f9ec128e6e6a482a2cb5c5422c56" +checksum = "2cc6181fd9a7492eef6fef1f33961e3695e4579b9872a6f7c83aee556666d4fe" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3316,28 +3292,38 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26c6ab57572f7a24a4985830b120de1594465e5d500f24afe89e16b4e833ef68" +checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.95" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65fc09f10666a9f147042251e0dda9c18f166ff7de300607007e96bdebc1068d" +checksum = "943aab3fdaaa029a6e0271b35ea10b72b943135afe9bffca82384098ad0e06a6" [[package]] name = "web-sys" -version = "0.3.72" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6488b90108c040df0fe62fa815cbdee25124641df01814dd7282749234c6112" +checksum = "04dd7223427d52553d3702c004d3b2fe07c148165faa56313cb00211e31c12bc" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "web-time" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a6580f308b1fad9207618087a65c04e7a10bc77e02c8e84e9b00dd4b12fa0bb" dependencies = [ "js-sys", "wasm-bindgen", @@ -3557,9 +3543,9 @@ checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" [[package]] name = "yoke" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c5b1314b079b0930c31e3af543d8ee1757b1951ae1e1565ec704403a7240ca5" +checksum = "120e6aef9aa629e3d4f52dc8cc43a015c7724194c97dfaf45180d2daf2b77f40" dependencies = [ "serde", "stable_deref_trait", @@ -3569,13 +3555,13 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28cc31741b18cb6f1d5ff12f5b7523e3d6eb0852bbbad19d73905511d9849b95" +checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "synstructure", ] @@ -3597,27 +3583,27 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] name = "zerofrom" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91ec111ce797d0e0784a1116d0ddcdbea84322cd79e5d5ad173daeba4f93ab55" +checksum = "cff3ee08c995dee1859d998dea82f7374f2826091dd9cd47def953cae446cd2e" dependencies = [ "zerofrom-derive", ] [[package]] name = "zerofrom-derive" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ea7b4a3637ea8669cedf0f1fd5c286a17f3de97b8dd5a70a6c167a1730e63a5" +checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", "synstructure", ] @@ -3640,7 +3626,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.87", + "syn 2.0.92", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index bd46cf0c9f..cf4921ebed 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -33,20 +33,21 @@ edition = "2021" rust-version = "1.79" [workspace.dependencies] -arrow = { version = "53.2.0", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { version = "53.2.0" } -arrow-buffer = { version = "53.2.0" } -arrow-data = { version = "53.2.0" } -arrow-schema = { version = "53.2.0" } -parquet = { version = "53.2.0", default-features = false, features = ["experimental"] } -datafusion = { version = "43.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { version = "43.0.0" } -datafusion-functions = { version = "43.0.0", features = ["crypto_expressions"] } -datafusion-functions-nested = { version = "43.0.0", default-features = false } -datafusion-expr = { version = "43.0.0", default-features = false } -datafusion-execution = { version = "43.0.0", default-features = false } -datafusion-physical-plan = { version = "43.0.0", default-features = false } -datafusion-physical-expr = { version = "43.0.0", default-features = false } +arrow = { version = "53.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "53.3.0" } +arrow-buffer = { version = "53.3.0" } +arrow-data = { version = "53.3.0" } +arrow-schema = { version = "53.3.0" } +parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false, features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/src/execution/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/expressions/bloom_filter_might_contain.rs index af6a5a47a3..b14fab62f6 100644 --- a/native/core/src/execution/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/expressions/bloom_filter_might_contain.rs @@ -19,26 +19,37 @@ use crate::{execution::util::spark_bloom_filter::SparkBloomFilter, parquet::data use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::physical_plan::ColumnarValue; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; -use std::{ - any::Any, - fmt::Display, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::hash::Hash; +use std::{any::Any, fmt::Display, sync::Arc}; /// A physical expression that checks if a value might be in a bloom filter. It corresponds to the /// Spark's `BloomFilterMightContain` expression. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct BloomFilterMightContain { pub bloom_filter_expr: Arc, pub value_expr: Arc, bloom_filter: Option, } +impl Hash for BloomFilterMightContain { + fn hash(&self, state: &mut H) { + self.bloom_filter_expr.hash(state); + self.value_expr.hash(state); + self.bloom_filter.hash(state); + } +} + +impl PartialEq for BloomFilterMightContain { + fn eq(&self, other: &Self) -> bool { + self.bloom_filter_expr.eq(&other.bloom_filter_expr) + && self.value_expr.eq(&other.value_expr) + && self.bloom_filter.eq(&other.bloom_filter) + } +} + impl Display for BloomFilterMightContain { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!( @@ -49,18 +60,6 @@ impl Display for BloomFilterMightContain { } } -impl PartialEq for BloomFilterMightContain { - fn eq(&self, _other: &dyn Any) -> bool { - down_cast_any_ref(_other) - .downcast_ref::() - .map(|other| { - self.bloom_filter_expr.eq(&other.bloom_filter_expr) - && self.value_expr.eq(&other.value_expr) - }) - .unwrap_or(false) - } -} - fn evaluate_bloom_filter( bloom_filter_expr: &Arc, ) -> Result> { @@ -141,11 +140,4 @@ impl PhysicalExpr for BloomFilterMightContain { Arc::clone(&children[1]), )?)) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.bloom_filter_expr.hash(&mut s); - self.value_expr.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/core/src/execution/expressions/subquery.rs b/native/core/src/execution/expressions/subquery.rs index 3eeb29c16e..d933a6096a 100644 --- a/native/core/src/execution/expressions/subquery.rs +++ b/native/core/src/execution/expressions/subquery.rs @@ -22,7 +22,6 @@ use crate::{ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema, TimeUnit}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{internal_err, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use jni::{ @@ -32,11 +31,11 @@ use jni::{ use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct Subquery { /// The ID of the execution context that owns this subquery. We use this ID to retrieve the /// subquery result. @@ -63,19 +62,6 @@ impl Display for Subquery { } } -impl PartialEq for Subquery { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.id.eq(&x.id) - && self.data_type.eq(&x.data_type) - && self.exec_context_id.eq(&x.exec_context_id) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for Subquery { fn as_any(&self) -> &dyn Any { self @@ -209,9 +195,4 @@ impl PhysicalExpr for Subquery { ) -> datafusion_common::Result> { Ok(self) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s) - } } diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 2c1a55f48f..09caf5e279 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -20,10 +20,7 @@ use arrow::datatypes::DataType as ArrowDataType; use arrow_array::RecordBatch; use datafusion::{ - execution::{ - disk_manager::DiskManagerConfig, - runtime_env::{RuntimeConfig, RuntimeEnv}, - }, + execution::{disk_manager::DiskManagerConfig, runtime_env::RuntimeEnv}, physical_plan::{display::DisplayableExecutionPlan, SendableRecordBatchStream}, prelude::{SessionConfig, SessionContext}, }; @@ -52,6 +49,7 @@ use crate::{ }; use datafusion_comet_proto::spark_operator::Operator; use datafusion_common::ScalarValue; +use datafusion_execution::runtime_env::RuntimeEnvBuilder; use futures::stream::StreamExt; use jni::{ objects::GlobalRef, @@ -188,7 +186,7 @@ fn prepare_datafusion_session_context( memory_fraction: f64, comet_task_memory_manager: Arc, ) -> CometResult { - let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); + let mut rt_config = RuntimeEnvBuilder::new().with_disk_manager(DiskManagerConfig::NewOs); // Check if we are using unified memory manager integrated with Spark. if use_unified_memory_manager { @@ -216,6 +214,7 @@ fn prepare_datafusion_session_context( &ScalarValue::Float64(Some(1.1)), ); + #[allow(deprecated)] let runtime = RuntimeEnv::try_new(rt_config)?; let mut session_ctx = SessionContext::new_with_config_rt(session_config, Arc::new(runtime)); diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index 8eeda8a5ad..cec00eb28c 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -30,6 +30,7 @@ use arrow_array::{ use arrow_data::transform::MutableArrayData; use arrow_schema::{ArrowError, DataType, Field, FieldRef, Schema, SchemaRef}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use datafusion::{execution::TaskContext, physical_expr::*, physical_plan::*}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result as DataFusionResult}; @@ -78,7 +79,8 @@ impl CopyExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&schema)), Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Self { diff --git a/native/core/src/execution/operators/expand.rs b/native/core/src/execution/operators/expand.rs index fb43a6e49f..f75822d408 100644 --- a/native/core/src/execution/operators/expand.rs +++ b/native/core/src/execution/operators/expand.rs @@ -17,10 +17,11 @@ use arrow_array::{RecordBatch, RecordBatchOptions}; use arrow_schema::SchemaRef; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::{ execution::TaskContext, physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, }, }; @@ -54,7 +55,8 @@ impl ExpandExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&schema)), Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Self { diff --git a/native/core/src/execution/operators/filter.rs b/native/core/src/execution/operators/filter.rs index d9a54712dc..eab30a3560 100644 --- a/native/core/src/execution/operators/filter.rs +++ b/native/core/src/execution/operators/filter.rs @@ -210,7 +210,8 @@ impl FilterExec { Ok(PlanProperties::new( eq_properties, input.output_partitioning().clone(), // Output Partitioning - input.execution_mode(), // Execution Mode + input.pipeline_behavior(), + input.boundedness(), )) } } diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index a297f87c1f..888cd2fdb5 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -28,6 +28,7 @@ use arrow_data::ffi::FFI_ArrowArray; use arrow_data::ArrayData; use arrow_schema::ffi::FFI_ArrowSchema; use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{ BaselineMetrics, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }; @@ -122,7 +123,8 @@ impl ScanExec { // The partitioning is not important because we are not using DataFusion's // query planner or optimizer Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Ok(Self { diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 0a74933547..5a35c62e33 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -94,7 +94,6 @@ use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter}, JoinType as DFJoinType, ScalarValue, }; -use datafusion_expr::expr::find_df_window_func; use datafusion_expr::{ AggregateUDF, ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, @@ -1515,10 +1514,7 @@ impl PhysicalPlanner { let builder = match datatype { DataType::Decimal128(_, _) => { - let func = AggregateUDF::new_from_impl(SumDecimal::try_new( - Arc::clone(&child), - datatype, - )?); + let func = AggregateUDF::new_from_impl(SumDecimal::try_new(datatype)?); AggregateExprBuilder::new(Arc::new(func), vec![child]) } _ => { @@ -1543,11 +1539,8 @@ impl PhysicalPlanner { let input_datatype = to_arrow_datatype(expr.sum_datatype.as_ref().unwrap()); let builder = match datatype { DataType::Decimal128(_, _) => { - let func = AggregateUDF::new_from_impl(AvgDecimal::new( - Arc::clone(&child), - datatype, - input_datatype, - )); + let func = + AggregateUDF::new_from_impl(AvgDecimal::new(datatype, input_datatype)); AggregateExprBuilder::new(Arc::new(func), vec![child]) } _ => { @@ -1556,11 +1549,7 @@ impl PhysicalPlanner { // failure since it should have already been checked at Spark side. let child: Arc = Arc::new(CastExpr::new(Arc::clone(&child), datatype.clone(), None)); - let func = AggregateUDF::new_from_impl(Avg::new( - Arc::clone(&child), - "avg", - datatype, - )); + let func = AggregateUDF::new_from_impl(Avg::new("avg", datatype)); AggregateExprBuilder::new(Arc::new(func), vec![child]) } }; @@ -1638,8 +1627,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Covariance::new( - Arc::clone(&child1), - Arc::clone(&child2), "covariance", datatype, StatsType::Sample, @@ -1655,8 +1642,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Covariance::new( - Arc::clone(&child1), - Arc::clone(&child2), "covariance_pop", datatype, StatsType::Population, @@ -1682,7 +1667,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Variance::new( - Arc::clone(&child), "variance", datatype, StatsType::Sample, @@ -1693,7 +1677,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Variance::new( - Arc::clone(&child), "variance_pop", datatype, StatsType::Population, @@ -1714,7 +1697,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Stddev::new( - Arc::clone(&child), "stddev", datatype, StatsType::Sample, @@ -1725,7 +1707,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Stddev::new( - Arc::clone(&child), "stddev_pop", datatype, StatsType::Population, @@ -1747,8 +1728,6 @@ impl PhysicalPlanner { self.create_expr(expr.child2.as_ref().unwrap(), Arc::clone(&schema))?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); let func = AggregateUDF::new_from_impl(Correlation::new( - Arc::clone(&child1), - Arc::clone(&child2), "correlation", datatype, expr.null_on_divide_by_zero, @@ -1935,7 +1914,7 @@ impl PhysicalPlanner { window_func_name, &window_args, partition_by, - sort_exprs, + &LexOrdering::new(sort_exprs.to_vec()), window_frame.into(), input_schema.as_ref(), false, // TODO: Ignore nulls @@ -1985,15 +1964,11 @@ impl PhysicalPlanner { /// Find DataFusion's built-in window function by name. fn find_df_window_function(&self, name: &str) -> Option { - if let Some(f) = find_df_window_func(name) { - Some(f) - } else { - let registry = &self.session_ctx.state(); - registry - .udaf(name) - .map(WindowFunctionDefinition::AggregateUDF) - .ok() - } + let registry = &self.session_ctx.state(); + registry + .udaf(name) + .map(WindowFunctionDefinition::AggregateUDF) + .ok() } /// Create a DataFusion physical partitioning from Spark physical partitioning @@ -2049,7 +2024,15 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - let data_type = func.inner().return_type(&coerced_types)?; + let data_type = match fun_name { + // workaround for https://github.com/apache/datafusion/issues/13716 + "datepart" => DataType::Int32, + _ => { + // TODO need to call `return_type_from_exprs` instead + #[allow(deprecated)] + func.inner().return_type(&coerced_types)? + } + }; (data_type, coerced_types) } diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 01117199eb..f3fa685b88 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -25,6 +25,7 @@ use arrow::{datatypes::*, ipc::writer::StreamWriter}; use async_trait::async_trait; use bytes::Buf; use crc32fast::Hasher; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::{ arrow::{ array::*, @@ -44,7 +45,7 @@ use datafusion::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }, }; @@ -191,7 +192,8 @@ impl ShuffleWriterExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), partitioning.clone(), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Ok(ShuffleWriterExec { diff --git a/native/core/src/execution/util/spark_bit_array.rs b/native/core/src/execution/util/spark_bit_array.rs index 6cfecc1bfb..3ac8b199bf 100644 --- a/native/core/src/execution/util/spark_bit_array.rs +++ b/native/core/src/execution/util/spark_bit_array.rs @@ -22,7 +22,7 @@ use std::iter::zip; /// A simple bit array implementation that simulates the behavior of Spark's BitArray which is /// used in the BloomFilter implementation. Some methods are not implemented as they are not /// required for the current use case. -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct SparkBitArray { data: Vec, bit_count: usize, diff --git a/native/core/src/execution/util/spark_bloom_filter.rs b/native/core/src/execution/util/spark_bloom_filter.rs index 2c3af16916..61245757cf 100644 --- a/native/core/src/execution/util/spark_bloom_filter.rs +++ b/native/core/src/execution/util/spark_bloom_filter.rs @@ -27,7 +27,7 @@ const SPARK_BLOOM_FILTER_VERSION_1: i32 = 1; /// A Bloom filter implementation that simulates the behavior of Spark's BloomFilter. /// It's not a complete implementation of Spark's BloomFilter, but just add the minimum /// methods to support mightContainsLong in the native side. -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct SparkBloomFilter { bits: SparkBitArray, num_hash_functions: u32, diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 27367d83e1..fc348f81bf 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -36,6 +36,7 @@ chrono = { workspace = true } datafusion = { workspace = true, features = ["parquet"] } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } +datafusion-expr-common = { workspace = true } datafusion-physical-expr = { workspace = true } chrono-tz = { workspace = true } num = { workspace = true } diff --git a/native/spark-expr/benches/aggregate.rs b/native/spark-expr/benches/aggregate.rs index 43194fdda2..051ac5eb62 100644 --- a/native/spark-expr/benches/aggregate.rs +++ b/native/spark-expr/benches/aggregate.rs @@ -66,7 +66,6 @@ fn criterion_benchmark(c: &mut Criterion) { group.bench_function("avg_decimal_comet", |b| { let comet_avg_decimal = Arc::new(AggregateUDF::new_from_impl(AvgDecimal::new( - Arc::clone(&c1), DataType::Decimal128(38, 10), DataType::Decimal128(38, 10), ))); @@ -96,7 +95,7 @@ fn criterion_benchmark(c: &mut Criterion) { group.bench_function("sum_decimal_comet", |b| { let comet_sum_decimal = Arc::new(AggregateUDF::new_from_impl( - SumDecimal::try_new(Arc::clone(&c1), DataType::Decimal128(38, 10)).unwrap(), + SumDecimal::try_new(DataType::Decimal128(38, 10)).unwrap(), )); b.to_async(&rt).iter(|| { black_box(agg_test( diff --git a/native/spark-expr/src/avg.rs b/native/spark-expr/src/avg.rs index 7820497d46..816440ac9a 100644 --- a/native/spark-expr/src/avg.rs +++ b/native/spark-expr/src/avg.rs @@ -27,11 +27,10 @@ use datafusion::logical_expr::{ type_coercion::aggregates::avg_return_type, Accumulator, EmitTo, GroupsAccumulator, Signature, }; use datafusion_common::{not_impl_err, Result, ScalarValue}; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; +use datafusion_physical_expr::expressions::format_state_name; use std::{any::Any, sync::Arc}; use arrow_array::ArrowNativeTypeOp; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; @@ -42,20 +41,19 @@ use DataType::*; pub struct Avg { name: String, signature: Signature, - expr: Arc, + // expr: Arc, input_data_type: DataType, result_data_type: DataType, } impl Avg { /// Create a new AVG aggregate function - pub fn new(expr: Arc, name: impl Into, data_type: DataType) -> Self { + pub fn new(name: impl Into, data_type: DataType) -> Self { let result_data_type = avg_return_type("avg", &data_type).unwrap(); Self { name: name.into(), signature: Signature::user_defined(Immutable), - expr, input_data_type: data_type, result_data_type, } @@ -139,20 +137,6 @@ impl AggregateUDFImpl for Avg { } } -impl PartialEq for Avg { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.input_data_type == x.input_data_type - && self.result_data_type == x.result_data_type - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the average #[derive(Debug, Default)] pub struct AvgAccumulator { diff --git a/native/spark-expr/src/avg_decimal.rs b/native/spark-expr/src/avg_decimal.rs index 163e1560b6..05fc28e583 100644 --- a/native/spark-expr/src/avg_decimal.rs +++ b/native/spark-expr/src/avg_decimal.rs @@ -25,14 +25,13 @@ use arrow_array::{ use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator, Signature}; use datafusion_common::{not_impl_err, Result, ScalarValue}; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; +use datafusion_physical_expr::expressions::format_state_name; use std::{any::Any, sync::Arc}; use crate::utils::is_valid_decimal_precision; use arrow_array::ArrowNativeTypeOp; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use datafusion::logical_expr::Volatility::Immutable; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::avg_return_type; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; @@ -43,17 +42,15 @@ use DataType::*; #[derive(Debug, Clone)] pub struct AvgDecimal { signature: Signature, - expr: Arc, sum_data_type: DataType, result_data_type: DataType, } impl AvgDecimal { /// Create a new AVG aggregate function - pub fn new(expr: Arc, result_type: DataType, sum_type: DataType) -> Self { + pub fn new(result_type: DataType, sum_type: DataType) -> Self { Self { signature: Signature::user_defined(Immutable), - expr, result_data_type: result_type, sum_data_type: sum_type, } @@ -156,19 +153,6 @@ impl AggregateUDFImpl for AvgDecimal { } } -impl PartialEq for AvgDecimal { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.sum_data_type == x.sum_data_type - && self.result_data_type == x.result_data_type - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the average for decimals #[derive(Debug)] struct AvgDecimalAccumulator { diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index 36234935e1..d7c31836ff 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -15,21 +15,16 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ array::*, datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{any::Any, sync::Arc}; macro_rules! compute_op { ($OPERAND:expr, $DT:ident) => {{ @@ -43,12 +38,24 @@ macro_rules! compute_op { } /// BitwiseNot expression -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct BitwiseNotExpr { /// Input expression arg: Arc, } +impl Hash for BitwiseNotExpr { + fn hash(&self, state: &mut H) { + self.arg.hash(state); + } +} + +impl PartialEq for BitwiseNotExpr { + fn eq(&self, other: &Self) -> bool { + self.arg.eq(&other.arg) + } +} + impl BitwiseNotExpr { /// Create new bitwise not expression pub fn new(arg: Arc) -> Self { @@ -114,21 +121,6 @@ impl PhysicalExpr for BitwiseNotExpr { ) -> Result> { Ok(Arc::new(BitwiseNotExpr::new(Arc::clone(&children[0])))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.arg.hash(&mut s); - self.hash(&mut s); - } -} - -impl PartialEq for BitwiseNotExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.arg.eq(&x.arg)) - .unwrap_or(false) - } } pub fn bitwise_not(arg: Arc) -> Result> { diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index d96bcbbdb6..6e0e0915cd 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -39,7 +39,6 @@ use arrow_array::builder::StringBuilder; use arrow_array::{DictionaryArray, StringArray, StructArray}; use arrow_schema::{DataType, Field, Schema}; use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, }; @@ -54,7 +53,7 @@ use std::str::FromStr; use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, num::Wrapping, sync::Arc, }; @@ -131,13 +130,29 @@ impl TimeStampInfo { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct Cast { pub child: Arc, pub data_type: DataType, pub cast_options: SparkCastOptions, } +impl PartialEq for Cast { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.data_type.eq(&other.data_type) + && self.cast_options.eq(&other.cast_options) + } +} + +impl Hash for Cast { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + self.cast_options.hash(state); + } +} + /// Determine if Comet supports a cast, taking options such as EvalMode and Timezone into account. pub fn cast_supported( from_type: &DataType, @@ -1681,19 +1696,6 @@ impl Display for Cast { } } -impl PartialEq for Cast { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.cast_options.eq(&x.cast_options) - && self.data_type.eq(&x.data_type) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for Cast { fn as_any(&self) -> &dyn Any { self @@ -1729,14 +1731,6 @@ impl PhysicalExpr for Cast { _ => internal_err!("Cast should have exactly one child"), } } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.cast_options.hash(&mut s); - self.hash(&mut s); - } } fn timestamp_parser( diff --git a/native/spark-expr/src/checkoverflow.rs b/native/spark-expr/src/checkoverflow.rs index e922171bd2..528bbd5d96 100644 --- a/native/spark-expr/src/checkoverflow.rs +++ b/native/spark-expr/src/checkoverflow.rs @@ -15,13 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Display, Formatter}, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ array::{as_primitive_array, Array, ArrayRef, Decimal128Array}, datatypes::{Decimal128Type, DecimalType}, @@ -29,21 +22,42 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Display, Formatter}, + sync::Arc, +}; /// This is from Spark `CheckOverflow` expression. Spark `CheckOverflow` expression rounds decimals /// to given scale and check if the decimals can fit in given precision. As `cast` kernel rounds /// decimals already, Comet `CheckOverflow` expression only checks if the decimals can fit in the /// precision. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct CheckOverflow { pub child: Arc, pub data_type: DataType, pub fail_on_error: bool, } +impl Hash for CheckOverflow { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + self.fail_on_error.hash(state); + } +} + +impl PartialEq for CheckOverflow { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.data_type.eq(&other.data_type) + && self.fail_on_error.eq(&other.fail_on_error) + } +} + impl CheckOverflow { pub fn new(child: Arc, data_type: DataType, fail_on_error: bool) -> Self { Self { @@ -64,19 +78,6 @@ impl Display for CheckOverflow { } } -impl PartialEq for CheckOverflow { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.data_type.eq(&x.data_type) - && self.fail_on_error.eq(&x.fail_on_error) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for CheckOverflow { fn as_any(&self) -> &dyn Any { self @@ -162,12 +163,4 @@ impl PhysicalExpr for CheckOverflow { self.fail_on_error, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.fail_on_error.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/correlation.rs b/native/spark-expr/src/correlation.rs index e5f36c6f95..e4ddab95de 100644 --- a/native/spark-expr/src/correlation.rs +++ b/native/spark-expr/src/correlation.rs @@ -26,13 +26,12 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// CORR aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -43,26 +42,16 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Correlation { name: String, signature: Signature, - expr1: Arc, - expr2: Arc, null_on_divide_by_zero: bool, } impl Correlation { - pub fn new( - expr1: Arc, - expr2: Arc, - name: impl Into, - data_type: DataType, - null_on_divide_by_zero: bool, - ) -> Self { + pub fn new(name: impl Into, data_type: DataType, null_on_divide_by_zero: bool) -> Self { // the result of correlation just support FLOAT64 data type. assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), - expr1, - expr2, null_on_divide_by_zero, } } @@ -131,20 +120,6 @@ impl AggregateUDFImpl for Correlation { } } -impl PartialEq for Correlation { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr1.eq(&x.expr1) - && self.expr2.eq(&x.expr2) - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - }) - .unwrap_or(false) - } -} - /// An accumulator to compute correlation #[derive(Debug)] pub struct CorrelationAccumulator { diff --git a/native/spark-expr/src/covariance.rs b/native/spark-expr/src/covariance.rs index 9166e39766..fa3563cdea 100644 --- a/native/spark-expr/src/covariance.rs +++ b/native/spark-expr/src/covariance.rs @@ -17,7 +17,7 @@ * under the License. */ -use std::{any::Any, sync::Arc}; +use std::any::Any; use arrow::{ array::{ArrayRef, Float64Array}, @@ -25,15 +25,14 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ downcast_value, unwrap_or_internal_err, DataFusionError, Result, ScalarValue, }; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// COVAR_SAMP and COVAR_POP aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -43,8 +42,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Covariance { name: String, signature: Signature, - expr1: Arc, - expr2: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -52,8 +49,6 @@ pub struct Covariance { impl Covariance { /// Create a new COVAR aggregate function pub fn new( - expr1: Arc, - expr2: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -64,8 +59,6 @@ impl Covariance { Self { name: name.into(), signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), - expr1, - expr2, stats_type, null_on_divide_by_zero, } @@ -126,21 +119,6 @@ impl AggregateUDFImpl for Covariance { } } -impl PartialEq for Covariance { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr1.eq(&x.expr1) - && self.expr2.eq(&x.expr2) - && self.stats_type == x.stats_type - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - }) - .unwrap_or(false) - } -} - /// An accumulator to compute covariance #[derive(Debug)] pub struct CovarianceAccumulator { diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index 193a90fb55..01c754ad6d 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -15,24 +15,19 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::Result; use datafusion_physical_expr::{expressions::CaseExpr, PhysicalExpr}; +use std::hash::Hash; +use std::{any::Any, sync::Arc}; /// IfExpr is a wrapper around CaseExpr, because `IF(a, b, c)` is semantically equivalent to /// `CASE WHEN a THEN b ELSE c END`. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct IfExpr { if_expr: Arc, true_expr: Arc, @@ -41,6 +36,23 @@ pub struct IfExpr { case_expr: Arc, } +impl Hash for IfExpr { + fn hash(&self, state: &mut H) { + self.if_expr.hash(state); + self.true_expr.hash(state); + self.false_expr.hash(state); + self.case_expr.hash(state); + } +} +impl PartialEq for IfExpr { + fn eq(&self, other: &Self) -> bool { + self.if_expr.eq(&other.if_expr) + && self.true_expr.eq(&other.true_expr) + && self.false_expr.eq(&other.false_expr) + && self.case_expr.eq(&other.case_expr) + } +} + impl std::fmt::Display for IfExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!( @@ -106,27 +118,6 @@ impl PhysicalExpr for IfExpr { Arc::clone(&children[2]), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.if_expr.hash(&mut s); - self.true_expr.hash(&mut s); - self.false_expr.hash(&mut s); - self.hash(&mut s); - } -} - -impl PartialEq for IfExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.if_expr.eq(&x.if_expr) - && self.true_expr.eq(&x.true_expr) - && self.false_expr.eq(&x.false_expr) - }) - .unwrap_or(false) - } } #[cfg(test)] diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index 7dc17b5688..fc31b11a0b 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -26,16 +26,15 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field, FieldRef, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{ cast::{as_int32_array, as_large_list_array, as_list_array}, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, }; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, sync::Arc, }; @@ -44,7 +43,7 @@ use std::{ // https://github.com/apache/spark/blob/master/common/utils/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java const MAX_ROUNDED_ARRAY_LENGTH: usize = 2147483632; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ListExtract { child: Arc, ordinal: Arc, @@ -53,6 +52,25 @@ pub struct ListExtract { fail_on_error: bool, } +impl Hash for ListExtract { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + self.default_value.hash(state); + self.one_based.hash(state); + self.fail_on_error.hash(state); + } +} +impl PartialEq for ListExtract { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.ordinal.eq(&other.ordinal) + && self.default_value.eq(&other.default_value) + && self.one_based.eq(&other.one_based) + && self.fail_on_error.eq(&other.fail_on_error) + } +} + impl ListExtract { pub fn new( child: Arc, @@ -176,16 +194,6 @@ impl PhysicalExpr for ListExtract { _ => internal_err!("ListExtract should have exactly two children"), } } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.default_value.hash(&mut s); - self.one_based.hash(&mut s); - self.fail_on_error.hash(&mut s); - self.hash(&mut s); - } } fn one_based_index(index: i32, len: usize) -> DataFusionResult> { @@ -267,33 +275,24 @@ impl Display for ListExtract { } } -impl PartialEq for ListExtract { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.ordinal.eq(&x.ordinal) - && (self.default_value.is_none() == x.default_value.is_none()) - && self - .default_value - .as_ref() - .zip(x.default_value.as_ref()) - .map(|(s, x)| s.eq(x)) - .unwrap_or(true) - && self.one_based.eq(&x.one_based) - && self.fail_on_error.eq(&x.fail_on_error) - }) - .unwrap_or(false) - } -} - -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct GetArrayStructFields { child: Arc, ordinal: usize, } +impl Hash for GetArrayStructFields { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + } +} +impl PartialEq for GetArrayStructFields { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) + } +} + impl GetArrayStructFields { pub fn new(child: Arc, ordinal: usize) -> Self { Self { child, ordinal } @@ -379,13 +378,6 @@ impl PhysicalExpr for GetArrayStructFields { _ => internal_err!("GetArrayStructFields should have exactly one child"), } } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.hash(&mut s); - } } fn get_array_struct_fields( @@ -417,16 +409,7 @@ impl Display for GetArrayStructFields { } } -impl PartialEq for GetArrayStructFields { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.ordinal.eq(&x.ordinal)) - .unwrap_or(false) - } -} - -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ArrayInsert { src_array_expr: Arc, pos_expr: Arc, @@ -434,6 +417,23 @@ pub struct ArrayInsert { legacy_negative_index: bool, } +impl Hash for ArrayInsert { + fn hash(&self, state: &mut H) { + self.src_array_expr.hash(state); + self.pos_expr.hash(state); + self.item_expr.hash(state); + self.legacy_negative_index.hash(state); + } +} +impl PartialEq for ArrayInsert { + fn eq(&self, other: &Self) -> bool { + self.src_array_expr.eq(&other.src_array_expr) + && self.pos_expr.eq(&other.pos_expr) + && self.item_expr.eq(&other.item_expr) + && self.legacy_negative_index.eq(&other.legacy_negative_index) + } +} + impl ArrayInsert { pub fn new( src_array_expr: Arc, @@ -555,15 +555,6 @@ impl PhysicalExpr for ArrayInsert { _ => internal_err!("ArrayInsert should have exactly three childrens"), } } - - fn dyn_hash(&self, _state: &mut dyn Hasher) { - let mut s = _state; - self.src_array_expr.hash(&mut s); - self.pos_expr.hash(&mut s); - self.item_expr.hash(&mut s); - self.legacy_negative_index.hash(&mut s); - self.hash(&mut s); - } } fn array_insert( @@ -694,20 +685,6 @@ impl Display for ArrayInsert { } } -impl PartialEq for ArrayInsert { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.src_array_expr.eq(&x.src_array_expr) - && self.pos_expr.eq(&x.pos_expr) - && self.item_expr.eq(&x.item_expr) - && self.legacy_negative_index.eq(&x.legacy_negative_index) - }) - .unwrap_or(false) - } -} - #[cfg(test)] mod test { use crate::list::{array_insert, list_extract, zero_based_index}; diff --git a/native/spark-expr/src/negative.rs b/native/spark-expr/src/negative.rs index 3d9063e783..7fb5089179 100644 --- a/native/spark-expr/src/negative.rs +++ b/native/spark-expr/src/negative.rs @@ -21,18 +21,14 @@ use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeT use arrow_array::RecordBatch; use arrow_buffer::IntervalDayTime; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, }; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::hash::Hash; +use std::{any::Any, sync::Arc}; pub fn create_negate_expr( expr: Arc, @@ -42,13 +38,26 @@ pub fn create_negate_expr( } /// Negative expression -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct NegativeExpr { /// Input expression arg: Arc, fail_on_error: bool, } +impl Hash for NegativeExpr { + fn hash(&self, state: &mut H) { + self.arg.hash(state); + self.fail_on_error.hash(state); + } +} + +impl PartialEq for NegativeExpr { + fn eq(&self, other: &Self) -> bool { + self.arg.eq(&other.arg) && self.fail_on_error.eq(&other.fail_on_error) + } +} + macro_rules! check_overflow { ($array:expr, $array_type:ty, $min_val:expr, $type_name:expr) => {{ let typed_array = $array @@ -204,11 +213,6 @@ impl PhysicalExpr for NegativeExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s); - } - /// Given the child interval of a NegativeExpr, it calculates the NegativeExpr's interval. /// It replaces the upper and lower bounds after multiplying them with -1. /// Ex: `(a, b]` => `[-b, -a)` @@ -255,12 +259,3 @@ impl PhysicalExpr for NegativeExpr { Ok(properties) } } - -impl PartialEq for NegativeExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.arg.eq(&x.arg)) - .unwrap_or(false) - } -} diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index c5331ad7bd..078ce4b5a4 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -15,13 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Display, Formatter}, - hash::{Hash, Hasher}, - sync::Arc, -}; - use arrow::{ array::{as_primitive_array, ArrayAccessor, ArrayIter, Float32Array, Float64Array}, datatypes::{ArrowNativeType, Float32Type, Float64Type}, @@ -29,15 +22,33 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Display, Formatter}, + sync::Arc, +}; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct NormalizeNaNAndZero { pub data_type: DataType, pub child: Arc, } +impl PartialEq for NormalizeNaNAndZero { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.data_type.eq(&other.data_type) + } +} + +impl Hash for NormalizeNaNAndZero { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + } +} + impl NormalizeNaNAndZero { pub fn new(data_type: DataType, child: Arc) -> Self { Self { data_type, child } @@ -89,13 +100,6 @@ impl PhysicalExpr for NormalizeNaNAndZero { Arc::clone(&children[0]), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.hash(&mut s); - } } fn eval_typed>(input: T) -> Vec> { @@ -120,15 +124,6 @@ impl Display for NormalizeNaNAndZero { } } -impl PartialEq for NormalizeNaNAndZero { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.data_type.eq(&x.data_type)) - .unwrap_or(false) - } -} - trait FloatDouble: ArrowNativeType { fn is_nan(&self) -> bool; fn nan(&self) -> Self; diff --git a/native/spark-expr/src/regexp.rs b/native/spark-expr/src/regexp.rs index c7626285a2..7f367a8bb9 100644 --- a/native/spark-expr/src/regexp.rs +++ b/native/spark-expr/src/regexp.rs @@ -21,7 +21,7 @@ use arrow_array::builder::BooleanBuilder; use arrow_array::types::Int32Type; use arrow_array::{Array, BooleanArray, DictionaryArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; @@ -53,6 +53,16 @@ impl Hash for RLike { } } +impl DynEq for RLike { + fn dyn_eq(&self, other: &dyn Any) -> bool { + if let Some(other) = other.downcast_ref::() { + self.pattern_str == other.pattern_str + } else { + false + } + } +} + impl RLike { pub fn try_new(child: Arc, pattern: &str) -> Result { Ok(Self { @@ -93,15 +103,6 @@ impl Display for RLike { } } -impl PartialEq for RLike { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.pattern_str.eq(&x.pattern_str)) - .unwrap_or(false) - } -} - impl PhysicalExpr for RLike { fn as_any(&self) -> &dyn Any { self @@ -161,10 +162,4 @@ impl PhysicalExpr for RLike { &self.pattern_str, )?)) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - use std::hash::Hash; - let mut s = state; - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/stddev.rs index 3cf604da0b..1ec5ffb69a 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -23,12 +23,12 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion_common::types::NativeType; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// STDDEV and STDDEV_SAMP (standard deviation) aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -39,7 +39,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Stddev { name: String, signature: Signature, - expr: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -47,7 +46,6 @@ pub struct Stddev { impl Stddev { /// Create a new STDDEV aggregate function pub fn new( - expr: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -57,8 +55,14 @@ impl Stddev { assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), - signature: Signature::coercible(vec![DataType::Float64], Volatility::Immutable), - expr, + signature: Signature::coercible( + vec![ + datafusion_expr_common::signature::TypeSignatureClass::Native(Arc::new( + NativeType::Float64, + )), + ], + Volatility::Immutable, + ), stats_type, null_on_divide_by_zero, } @@ -121,20 +125,6 @@ impl AggregateUDFImpl for Stddev { } } -impl PartialEq for Stddev { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr.eq(&x.expr) - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - && self.stats_type == x.stats_type - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the standard deviation #[derive(Debug)] pub struct StddevAccumulator { diff --git a/native/spark-expr/src/strings.rs b/native/spark-expr/src/strings.rs index a8aab6aee9..c2706b5896 100644 --- a/native/spark-expr/src/strings.rs +++ b/native/spark-expr/src/strings.rs @@ -27,19 +27,18 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; macro_rules! make_predicate_function { ($name: ident, $kernel: ident, $str_scalar_kernel: ident) => { - #[derive(Debug, Hash)] + #[derive(Debug, Eq)] pub struct $name { left: Arc, right: Arc, @@ -57,12 +56,16 @@ macro_rules! make_predicate_function { } } - impl PartialEq for $name { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.left.eq(&x.left) && self.right.eq(&x.right)) - .unwrap_or(false) + impl Hash for $name { + fn hash(&self, state: &mut H) { + self.left.hash(state); + self.right.hash(state); + } + } + + impl PartialEq for $name { + fn eq(&self, other: &Self) -> bool { + self.left.eq(&other.left) && self.right.eq(&other.right) } } @@ -122,13 +125,6 @@ macro_rules! make_predicate_function { children[1].clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.left.hash(&mut s); - self.right.hash(&mut s); - self.hash(&mut s); - } } }; } @@ -141,18 +137,43 @@ make_predicate_function!(EndsWith, ends_with_dyn, ends_with_utf8_scalar_dyn); make_predicate_function!(Contains, contains_dyn, contains_utf8_scalar_dyn); -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct SubstringExpr { pub child: Arc, pub start: i64, pub len: u64, } -#[derive(Debug, Hash)] +impl Hash for SubstringExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.start.hash(state); + self.len.hash(state); + } +} + +impl PartialEq for SubstringExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.start.eq(&other.start) && self.len.eq(&other.len) + } +} +#[derive(Debug, Eq)] pub struct StringSpaceExpr { pub child: Arc, } +impl Hash for StringSpaceExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + } +} + +impl PartialEq for StringSpaceExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + } +} + impl SubstringExpr { pub fn new(child: Arc, start: i64, len: u64) -> Self { Self { child, start, len } @@ -181,15 +202,6 @@ impl Display for StringSpaceExpr { } } -impl PartialEq for SubstringExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.start.eq(&x.start) && self.len.eq(&x.len)) - .unwrap_or(false) - } -} - impl PhysicalExpr for SubstringExpr { fn as_any(&self) -> &dyn Any { self @@ -231,23 +243,6 @@ impl PhysicalExpr for SubstringExpr { self.len, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.start.hash(&mut s); - self.len.hash(&mut s); - self.hash(&mut s); - } -} - -impl PartialEq for StringSpaceExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child)) - .unwrap_or(false) - } } impl PhysicalExpr for StringSpaceExpr { @@ -292,10 +287,4 @@ impl PhysicalExpr for StringSpaceExpr { ) -> datafusion_common::Result> { Ok(Arc::new(StringSpaceExpr::new(Arc::clone(&children[0])))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index cda8246d90..7cc49e4281 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -19,17 +19,16 @@ use arrow::record_batch::RecordBatch; use arrow_array::{Array, StructArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct CreateNamedStruct { values: Vec>, names: Vec, @@ -95,13 +94,6 @@ impl PhysicalExpr for CreateNamedStruct { self.names.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.values.hash(&mut s); - self.names.hash(&mut s); - self.hash(&mut s); - } } impl Display for CreateNamedStruct { @@ -114,29 +106,24 @@ impl Display for CreateNamedStruct { } } -impl PartialEq for CreateNamedStruct { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.values - .iter() - .zip(x.values.iter()) - .all(|(a, b)| a.eq(b)) - && self.values.len() == x.values.len() - && self.names.iter().zip(x.names.iter()).all(|(a, b)| a.eq(b)) - && self.names.len() == x.names.len() - }) - .unwrap_or(false) - } -} - -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct GetStructField { child: Arc, ordinal: usize, } +impl Hash for GetStructField { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + } +} +impl PartialEq for GetStructField { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) + } +} + impl GetStructField { pub fn new(child: Arc, ordinal: usize) -> Self { Self { child, ordinal } @@ -203,13 +190,6 @@ impl PhysicalExpr for GetStructField { self.ordinal, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.hash(&mut s); - } } impl Display for GetStructField { @@ -222,15 +202,6 @@ impl Display for GetStructField { } } -impl PartialEq for GetStructField { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.ordinal.eq(&x.ordinal)) - .unwrap_or(false) - } -} - #[cfg(test)] mod test { use super::CreateNamedStruct; diff --git a/native/spark-expr/src/sum_decimal.rs b/native/spark-expr/src/sum_decimal.rs index ab142aee69..f3f34d9bfa 100644 --- a/native/spark-expr/src/sum_decimal.rs +++ b/native/spark-expr/src/sum_decimal.rs @@ -25,20 +25,16 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, Result as DFResult, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF, Signature}; -use datafusion_physical_expr::PhysicalExpr; use std::{any::Any, ops::BitAnd, sync::Arc}; #[derive(Debug)] pub struct SumDecimal { /// Aggregate function signature signature: Signature, - /// The expression that provides the input decimal values to be summed - expr: Arc, /// The data type of the SUM result. This will always be a decimal type /// with the same precision and scale as specified in this struct result_type: DataType, @@ -49,7 +45,7 @@ pub struct SumDecimal { } impl SumDecimal { - pub fn try_new(expr: Arc, data_type: DataType) -> DFResult { + pub fn try_new(data_type: DataType) -> DFResult { // The `data_type` is the SUM result type passed from Spark side let (precision, scale) = match data_type { DataType::Decimal128(p, s) => (p, s), @@ -61,7 +57,6 @@ impl SumDecimal { }; Ok(Self { signature: Signature::user_defined(Immutable), - expr, result_type: data_type, precision, scale, @@ -132,20 +127,6 @@ impl AggregateUDFImpl for SumDecimal { } } -impl PartialEq for SumDecimal { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - // note that we do not compare result_type because this - // is guaranteed to match if the precision and scale - // match - self.precision == x.precision && self.scale == x.scale && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - #[derive(Debug)] struct SumDecimalAccumulator { sum: i128, @@ -491,13 +472,13 @@ mod tests { use datafusion_common::Result; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; - use datafusion_physical_expr::expressions::{Column, Literal}; + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_expr::PhysicalExpr; use futures::StreamExt; #[test] fn invalid_data_type() { - let expr = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))); - assert!(SumDecimal::try_new(expr, DataType::Int32).is_err()); + assert!(SumDecimal::try_new(DataType::Int32).is_err()); } #[tokio::test] @@ -518,7 +499,6 @@ mod tests { Arc::new(MemoryExec::try_new(partitions, Arc::clone(&schema), None).unwrap()); let aggregate_udf = Arc::new(AggregateUDF::new_from_impl(SumDecimal::try_new( - Arc::clone(&c1), data_type.clone(), )?)); diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index 91953dd600..fb549f9ce8 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -15,36 +15,45 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, - sync::Arc, -}; - +use crate::utils::array_with_timezone; use arrow::{ compute::{date_part, DatePart}, record_batch::RecordBatch, }; use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; - -use crate::utils::array_with_timezone; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; use crate::kernels::temporal::{ date_trunc_array_fmt_dyn, date_trunc_dyn, timestamp_trunc_array_fmt_dyn, timestamp_trunc_dyn, }; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct HourExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl Hash for HourExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for HourExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + impl HourExpr { pub fn new(child: Arc, timezone: String) -> Self { HourExpr { child, timezone } @@ -61,15 +70,6 @@ impl Display for HourExpr { } } -impl PartialEq for HourExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for HourExpr { fn as_any(&self) -> &dyn Any { self @@ -123,22 +123,27 @@ impl PhysicalExpr for HourExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct MinuteExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl Hash for MinuteExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for MinuteExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + impl MinuteExpr { pub fn new(child: Arc, timezone: String) -> Self { MinuteExpr { child, timezone } @@ -155,15 +160,6 @@ impl Display for MinuteExpr { } } -impl PartialEq for MinuteExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for MinuteExpr { fn as_any(&self) -> &dyn Any { self @@ -217,22 +213,27 @@ impl PhysicalExpr for MinuteExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct SecondExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl Hash for SecondExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for SecondExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + impl SecondExpr { pub fn new(child: Arc, timezone: String) -> Self { SecondExpr { child, timezone } @@ -249,15 +250,6 @@ impl Display for SecondExpr { } } -impl PartialEq for SecondExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for SecondExpr { fn as_any(&self) -> &dyn Any { self @@ -311,16 +303,9 @@ impl PhysicalExpr for SecondExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct DateTruncExpr { /// An array with DataType::Date32 child: Arc, @@ -328,6 +313,18 @@ pub struct DateTruncExpr { format: Arc, } +impl Hash for DateTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); + } +} +impl PartialEq for DateTruncExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.format.eq(&other.format) + } +} + impl DateTruncExpr { pub fn new(child: Arc, format: Arc) -> Self { DateTruncExpr { child, format } @@ -344,15 +341,6 @@ impl Display for DateTruncExpr { } } -impl PartialEq for DateTruncExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.format.eq(&x.format)) - .unwrap_or(false) - } -} - impl PhysicalExpr for DateTruncExpr { fn as_any(&self) -> &dyn Any { self @@ -398,16 +386,9 @@ impl PhysicalExpr for DateTruncExpr { Arc::clone(&self.format), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.format.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct TimestampTruncExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -422,6 +403,21 @@ pub struct TimestampTruncExpr { timezone: String, } +impl Hash for TimestampTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for TimestampTruncExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.format.eq(&other.format) + && self.timezone.eq(&other.timezone) + } +} + impl TimestampTruncExpr { pub fn new( child: Arc, @@ -446,19 +442,6 @@ impl Display for TimestampTruncExpr { } } -impl PartialEq for TimestampTruncExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.format.eq(&x.format) - && self.timezone.eq(&x.timezone) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for TimestampTruncExpr { fn as_any(&self) -> &dyn Any { self @@ -524,12 +507,4 @@ impl PhysicalExpr for TimestampTruncExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.format.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 1f68eb8606..91b46c6f04 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -29,11 +29,11 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; use std::any::Any; use std::fmt::{Debug, Display, Formatter}; -use std::hash::{Hash, Hasher}; +use std::hash::Hash; use std::sync::Arc; /// to_json function -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ToJson { /// The input to convert to JSON expr: Arc, @@ -41,6 +41,18 @@ pub struct ToJson { timezone: String, } +impl Hash for ToJson { + fn hash(&self, state: &mut H) { + self.expr.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for ToJson { + fn eq(&self, other: &Self) -> bool { + self.expr.eq(&other.expr) && self.timezone.eq(&other.timezone) + } +} + impl ToJson { pub fn new(expr: Arc, timezone: &str) -> Self { Self { @@ -101,13 +113,6 @@ impl PhysicalExpr for ToJson { &self.timezone, ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.expr.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } /// Convert an array into a JSON value string representation diff --git a/native/spark-expr/src/unbound.rs b/native/spark-expr/src/unbound.rs index a6babd0f7e..14f68c9cd6 100644 --- a/native/spark-expr/src/unbound.rs +++ b/native/spark-expr/src/unbound.rs @@ -17,15 +17,10 @@ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::physical_plan::ColumnarValue; use datafusion_common::{internal_err, Result}; use datafusion_physical_expr::PhysicalExpr; -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::{hash::Hash, sync::Arc}; /// This is similar to `UnKnownColumn` in DataFusion, but it has data type. /// This is only used when the column is not bound to a schema, for example, the @@ -93,18 +88,4 @@ impl PhysicalExpr for UnboundColumn { ) -> Result> { Ok(self) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s); - } -} - -impl PartialEq for UnboundColumn { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self == x) - .unwrap_or(false) - } } diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index 4370d89ff4..e71d713f59 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -15,20 +15,19 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, sync::Arc}; +use std::any::Any; use arrow::{ array::{ArrayRef, Float64Array}, datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, Signature}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// VAR_SAMP and VAR_POP aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -39,7 +38,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Variance { name: String, signature: Signature, - expr: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -47,7 +45,6 @@ pub struct Variance { impl Variance { /// Create a new VARIANCE aggregate function pub fn new( - expr: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -58,7 +55,6 @@ impl Variance { Self { name: name.into(), signature: Signature::numeric(1, Immutable), - expr, stats_type, null_on_divide_by_zero, } @@ -118,17 +114,6 @@ impl AggregateUDFImpl for Variance { } } -impl PartialEq for Variance { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name && self.expr.eq(&x.expr) && self.stats_type == x.stats_type - }) - .unwrap_or(false) - } -} - /// An accumulator to compute variance #[derive(Debug)] pub struct VarianceAccumulator { From 4f8ce757b2832730e3d6974848cf5d3d89072283 Mon Sep 17 00:00:00 2001 From: Dharan Aditya Date: Fri, 3 Jan 2025 01:13:02 +0530 Subject: [PATCH 47/47] feat: add support for array_contains expression (#1163) * feat: add support for array_contains expression * test: add unit test for array_contains function * Removes unnecessary case expression for handling null values * chore: Move more expressions from core crate to spark-expr crate (#1152) * move aggregate expressions to spark-expr crate * move more expressions * move benchmark * normalize_nan * bitwise not * comet scalar funcs * update bench imports * remove dead code (#1155) * fix: Spark 4.0-preview1 SPARK-47120 (#1156) ## Which issue does this PR close? Part of https://github.com/apache/datafusion-comet/issues/372 and https://github.com/apache/datafusion-comet/issues/551 ## Rationale for this change To be ready for Spark 4.0 ## What changes are included in this PR? This PR fixes the new test SPARK-47120 added in Spark 4.0 ## How are these changes tested? tests enabled * chore: Move string kernels and expressions to spark-expr crate (#1164) * Move string kernels and expressions to spark-expr crate * remove unused hash kernel * remove unused dependencies * chore: Move remaining expressions to spark-expr crate + some minor refactoring (#1165) * move CheckOverflow to spark-expr crate * move NegativeExpr to spark-expr crate * move UnboundColumn to spark-expr crate * move ExpandExec from execution::datafusion::operators to execution::operators * refactoring to remove datafusion subpackage * update imports in benches * fix * fix * chore: Add ignored tests for reading complex types from Parquet (#1167) * Add ignored tests for reading structs from Parquet * add basic map test * add tests for Map and Array * feat: Add Spark-compatible implementation of SchemaAdapterFactory (#1169) * Add Spark-compatible SchemaAdapterFactory implementation * remove prototype code * fix * refactor * implement more cast logic * implement more cast logic * add basic test * improve test * cleanup * fmt * add support for casting unsigned int to signed int * clippy * address feedback * fix test * fix: Document enabling comet explain plan usage in Spark (4.0) (#1176) * test: enabling Spark tests with offHeap requirement (#1177) ## Which issue does this PR close? ## Rationale for this change After https://github.com/apache/datafusion-comet/pull/1062 We have not running Spark tests for native execution ## What changes are included in this PR? Removed the off heap requirement for testing ## How are these changes tested? Bringing back Spark tests for native execution * feat: Improve shuffle metrics (second attempt) (#1175) * improve shuffle metrics * docs * more metrics * refactor * address feedback * fix: stddev_pop should not directly return 0.0 when count is 1.0 (#1184) * add test * fix * fix * fix * feat: Make native shuffle compression configurable and respect `spark.shuffle.compress` (#1185) * Make shuffle compression codec and level configurable * remove lz4 references * docs * update comment * clippy * fix benches * clippy * clippy * disable test for miri * remove lz4 reference from proto * minor: move shuffle classes from common to spark (#1193) * minor: refactor decodeBatches to make private in broadcast exchange (#1195) * minor: refactor prepare_output so that it does not require an ExecutionContext (#1194) * fix: fix missing explanation for then branch in case when (#1200) * minor: remove unused source files (#1202) * chore: Upgrade to DataFusion 44.0.0-rc2 (#1154) * move aggregate expressions to spark-expr crate * move more expressions * move benchmark * normalize_nan * bitwise not * comet scalar funcs * update bench imports * save * save * save * remove unused imports * clippy * implement more hashers * implement Hash and PartialEq * implement Hash and PartialEq * implement Hash and PartialEq * benches * fix ScalarUDFImpl.return_type failure * exclude test from miri * ignore correct test * ignore another test * remove miri checks * use return_type_from_exprs * Revert "use return_type_from_exprs" This reverts commit febc1f1ec1301f9b359fc23ad6a117224fce35b7. * use DF main branch * hacky workaround for regression in ScalarUDFImpl.return_type * fix repo url * pin to revision * bump to latest rev * bump to latest DF rev * bump DF to rev 9f530dd * add Cargo.lock * bump DF version * no default features * Revert "remove miri checks" This reverts commit 4638fe3aa5501966cd5d8b53acf26c698b10b3c9. * Update pin to DataFusion e99e02b9b9093ceb0c13a2dd32a2a89beba47930 * update pin * Update Cargo.toml Bump to 44.0.0-rc2 * update cargo lock * revert miri change --------- Co-authored-by: Andrew Lamb * update UT Signed-off-by: Dharan Aditya * fix typo in UT Signed-off-by: Dharan Aditya --------- Signed-off-by: Dharan Aditya Co-authored-by: Andy Grove Co-authored-by: KAZUYUKI TANIMURA Co-authored-by: Parth Chandra Co-authored-by: Liang-Chi Hsieh Co-authored-by: Raz Luvaton <16746759+rluvaton@users.noreply.github.com> Co-authored-by: Andrew Lamb --- native/core/src/execution/planner.rs | 15 +++++++++++++++ native/proto/src/proto/expr.proto | 1 + .../org/apache/comet/serde/QueryPlanSerde.scala | 6 ++++++ .../org/apache/comet/CometExpressionSuite.scala | 12 ++++++++++++ 4 files changed, 34 insertions(+) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5a35c62e33..c40e2e73f9 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -98,6 +98,7 @@ use datafusion_expr::{ AggregateUDF, ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; +use datafusion_functions_nested::array_has::ArrayHas; use datafusion_physical_expr::expressions::{Literal, StatsType}; use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::LexOrdering; @@ -719,6 +720,20 @@ impl PhysicalPlanner { expr.legacy_negative_index, ))) } + ExprStruct::ArrayContains(expr) => { + let src_array_expr = + self.create_expr(expr.left.as_ref().unwrap(), Arc::clone(&input_schema))?; + let key_expr = + self.create_expr(expr.right.as_ref().unwrap(), Arc::clone(&input_schema))?; + let args = vec![Arc::clone(&src_array_expr), key_expr]; + let array_has_expr = Arc::new(ScalarFunctionExpr::new( + "array_has", + Arc::new(ScalarUDF::new_from_impl(ArrayHas::new())), + args, + DataType::Boolean, + )); + Ok(array_has_expr) + } expr => Err(ExecutionError::GeneralError(format!( "Not implemented: {:?}", expr diff --git a/native/proto/src/proto/expr.proto b/native/proto/src/proto/expr.proto index 7a8ea78d57..e76ecdccf1 100644 --- a/native/proto/src/proto/expr.proto +++ b/native/proto/src/proto/expr.proto @@ -84,6 +84,7 @@ message Expr { GetArrayStructFields get_array_struct_fields = 57; BinaryExpr array_append = 58; ArrayInsert array_insert = 59; + BinaryExpr array_contains = 60; } } diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 518fa06858..dc081b196b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2266,6 +2266,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim withInfo(expr, "unsupported arguments for GetArrayStructFields", child) None } + case expr if expr.prettyName == "array_contains" => + createBinaryExpr( + expr.children(0), + expr.children(1), + inputs, + (builder, binaryExpr) => builder.setArrayContains(binaryExpr)) case _ if expr.prettyName == "array_append" => createBinaryExpr( expr.children(0), diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index cce7cb20a1..36d3706506 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -2517,4 +2517,16 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { checkSparkAnswer(df.select("arrUnsupportedArgs")) } } + + test("array_contains") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = false, n = 10000) + spark.read.parquet(path.toString).createOrReplaceTempView("t1"); + checkSparkAnswerAndOperator( + spark.sql("SELECT array_contains(array(_2, _3, _4), _2) FROM t1")) + checkSparkAnswerAndOperator( + spark.sql("SELECT array_contains((CASE WHEN _2 =_3 THEN array(_4) END), _4) FROM t1")); + } + } }