diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 9e594804f..1cc42f62e 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -28,6 +28,8 @@ import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; import org.apache.arrow.c.CometSchemaImporter; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.dictionary.Dictionary; import org.apache.arrow.vector.types.pojo.DictionaryEncoding; @@ -50,6 +52,7 @@ public class ColumnReader extends AbstractColumnReader { protected static final Logger LOG = LoggerFactory.getLogger(ColumnReader.class); + protected final BufferAllocator ALLOCATOR = new RootAllocator(); /** * The current Comet vector holding all the values read by this column reader. Owned by this @@ -87,6 +90,9 @@ public class ColumnReader extends AbstractColumnReader { private final CometSchemaImporter importer; + private ArrowArray array = null; + private ArrowSchema schema = null; + public ColumnReader( DataType type, ColumnDescriptor descriptor, @@ -201,53 +207,56 @@ public CometDecodedVector loadVector() { boolean isUuid = logicalTypeAnnotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; - long[] addresses = Native.currentBatch(nativeHandle); + array = ArrowArray.allocateNew(ALLOCATOR); + schema = ArrowSchema.allocateNew(ALLOCATOR); - try (ArrowArray array = ArrowArray.wrap(addresses[0]); - ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { - FieldVector vector = importer.importVector(array, schema); + long arrayAddr = array.memoryAddress(); + long schemaAddr = schema.memoryAddress(); - DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); + Native.currentBatch(nativeHandle, arrayAddr, schemaAddr); - CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); + FieldVector vector = importer.importVector(array, schema); - // Update whether the current vector contains any null values. This is used in the following - // batch(s) to determine whether we can skip loading the native vector. - hadNull = cometVector.hasNull(); + DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); - if (dictionaryEncoding == null) { - if (dictionary != null) { - // This means the column was using dictionary encoding but now has fall-back to plain - // encoding, on the native side. Setting 'dictionary' to null here, so we can use it as - // a condition to check if we can re-use vector later. - dictionary = null; - } - // Either the column is not dictionary encoded, or it was using dictionary encoding but - // a new data page has switched back to use plain encoding. For both cases we should - // return plain vector. - currentVector = cometVector; - return currentVector; - } + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); + + // Update whether the current vector contains any null values. This is used in the following + // batch(s) to determine whether we can skip loading the native vector. + hadNull = cometVector.hasNull(); - // We should already re-initiate `CometDictionary` here because `Data.importVector` API will - // release the previous dictionary vector and create a new one. - Dictionary arrowDictionary = importer.getProvider().lookup(dictionaryEncoding.getId()); - CometPlainVector dictionaryVector = - new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); + if (dictionaryEncoding == null) { if (dictionary != null) { - dictionary.setDictionaryVector(dictionaryVector); - } else { - dictionary = new CometDictionary(dictionaryVector); + // This means the column was using dictionary encoding but now has fall-back to plain + // encoding, on the native side. Setting 'dictionary' to null here, so we can use it as + // a condition to check if we can re-use vector later. + dictionary = null; } - - currentVector = - new CometDictionaryVector( - cometVector, dictionary, importer.getProvider(), useDecimal128, false, isUuid); - - currentVector = - new CometDictionaryVector(cometVector, dictionary, importer.getProvider(), useDecimal128); + // Either the column is not dictionary encoded, or it was using dictionary encoding but + // a new data page has switched back to use plain encoding. For both cases we should + // return plain vector. + currentVector = cometVector; return currentVector; } + + // We should already re-initiate `CometDictionary` here because `Data.importVector` API will + // release the previous dictionary vector and create a new one. + Dictionary arrowDictionary = importer.getProvider().lookup(dictionaryEncoding.getId()); + CometPlainVector dictionaryVector = + new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); + if (dictionary != null) { + dictionary.setDictionaryVector(dictionaryVector); + } else { + dictionary = new CometDictionary(dictionaryVector); + } + + currentVector = + new CometDictionaryVector( + cometVector, dictionary, importer.getProvider(), useDecimal128, false, isUuid); + + currentVector = + new CometDictionaryVector(cometVector, dictionary, importer.getProvider(), useDecimal128); + return currentVector; } protected void readPage() { diff --git a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java index b8722ca78..13b90e256 100644 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -34,8 +34,12 @@ /** A metadata column reader that can be extended by {@link RowIndexColumnReader} etc. */ public class MetadataColumnReader extends AbstractColumnReader { private final BufferAllocator allocator = new RootAllocator(); + private CometVector vector; + private ArrowArray array = null; + private ArrowSchema schema = null; + public MetadataColumnReader(DataType type, ColumnDescriptor descriptor, boolean useDecimal128) { // TODO: should we handle legacy dates & timestamps for metadata columns? super(type, descriptor, useDecimal128, false); @@ -50,13 +54,17 @@ public void setBatchSize(int batchSize) { @Override public void readBatch(int total) { if (vector == null) { - long[] addresses = Native.currentBatch(nativeHandle); - try (ArrowArray array = ArrowArray.wrap(addresses[0]); - ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { - FieldVector fieldVector = Data.importVector(allocator, array, schema, null); - vector = new CometPlainVector(fieldVector, useDecimal128); - } + array = ArrowArray.allocateNew(allocator); + schema = ArrowSchema.allocateNew(allocator); + + long arrayAddr = array.memoryAddress(); + long schemaAddr = schema.memoryAddress(); + + Native.currentBatch(nativeHandle, arrayAddr, schemaAddr); + FieldVector fieldVector = Data.importVector(allocator, array, schema, null); + vector = new CometPlainVector(fieldVector, useDecimal128); } + vector.setNumValues(total); } diff --git a/common/src/main/java/org/apache/comet/parquet/Native.java b/common/src/main/java/org/apache/comet/parquet/Native.java index f4820fedf..1e666652e 100644 --- a/common/src/main/java/org/apache/comet/parquet/Native.java +++ b/common/src/main/java/org/apache/comet/parquet/Native.java @@ -192,10 +192,10 @@ public static native void setPageV2( * Returns the current batch constructed via 'readBatch' * * @param handle the handle to the native Parquet column reader - * @return a long array with 2 elements, the first is the address to native Arrow array, and the - * second is the address to the Arrow schema. + * @param arrayAddr the memory address to the ArrowArray struct + * @param schemaAddr the memory address to the ArrowSchema struct */ - public static native long[] currentBatch(long handle); + public static native void currentBatch(long handle, long arrayAddr, long schemaAddr); /** Set methods to set a constant value for the reader, so it'll return constant vectors */ public static native void setNull(long handle); diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 6833aefff..7450d27a6 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -73,7 +73,7 @@ object CometConf extends ShimCometConf { "Whether to enable native scans. When this is turned on, Spark will use Comet to " + "read supported data sources (currently only Parquet is supported natively). Note " + "that to enable native vectorized execution, both this config and " + - "'spark.comet.exec.enabled' need to be enabled. By default, this config is true.") + "'spark.comet.exec.enabled' need to be enabled.") .booleanConf .createWithDefault(true) @@ -82,7 +82,7 @@ object CometConf extends ShimCometConf { .doc( "Whether to enable Comet's parallel reader for Parquet files. The parallel reader reads " + "ranges of consecutive data in a file in parallel. It is faster for large files and " + - "row groups but uses more resources. The parallel reader is enabled by default.") + "row groups but uses more resources.") .booleanConf .createWithDefault(true) @@ -98,7 +98,7 @@ object CometConf extends ShimCometConf { .doc( "When enabled the parallel reader will try to merge ranges of data that are separated " + "by less than 'comet.parquet.read.io.mergeRanges.delta' bytes. Longer continuous reads " + - "are faster on cloud storage. The default behavior is to merge consecutive ranges.") + "are faster on cloud storage.") .booleanConf .createWithDefault(true) @@ -115,7 +115,7 @@ object CometConf extends ShimCometConf { .doc("In the parallel reader, if the read ranges submitted are skewed in sizes, this " + "option will cause the reader to break up larger read ranges into smaller ranges to " + "reduce the skew. This will result in a slightly larger number of connections opened to " + - "the file system but may give improved performance. The option is off by default.") + "the file system but may give improved performance.") .booleanConf .createWithDefault(false) @@ -153,7 +153,7 @@ object CometConf extends ShimCometConf { "native space. Note: each operator is associated with a separate config in the " + "format of 'spark.comet.exec..enabled' at the moment, and both the " + "config and this need to be turned on, in order for the operator to be executed in " + - "native. By default, this config is true.") + "native.") .booleanConf .createWithDefault(true) @@ -215,7 +215,7 @@ object CometConf extends ShimCometConf { "spark.comet.memory.overhead.factor") .doc( "Fraction of executor memory to be allocated as additional non-heap memory per executor " + - "process for Comet. Default value is 0.2.") + "process for Comet.") .doubleConf .checkValue( factor => factor > 0, @@ -247,8 +247,7 @@ object CometConf extends ShimCometConf { "is enabled. Available modes are 'native', 'jvm', and 'auto'. " + "'native' is for native shuffle which has best performance in general. " + "'jvm' is for jvm-based columnar shuffle which has higher coverage than native shuffle. " + - "'auto' is for Comet to choose the best shuffle mode based on the query plan. " + - "By default, this config is 'auto'.") + "'auto' is for Comet to choose the best shuffle mode based on the query plan.") .internal() .stringConf .transform(_.toLowerCase(Locale.ROOT)) @@ -258,8 +257,8 @@ object CometConf extends ShimCometConf { val COMET_EXEC_BROADCAST_FORCE_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.broadcast.enabled") .doc( - "Whether to force enabling broadcasting for Comet native operators. By default, " + - "this config is false. Comet broadcast feature will be enabled automatically by " + + "Whether to force enabling broadcasting for Comet native operators. " + + "Comet broadcast feature will be enabled automatically by " + "Comet extension. But for unit tests, we need this feature to force enabling it " + "for invalid cases. So this config is only used for unit test.") .internal() @@ -280,27 +279,26 @@ object CometConf extends ShimCometConf { .stringConf .createWithDefault("zstd") - 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. By default, this config " + - "is false.") - .booleanConf - .createWithDefault(false) + 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.") + .booleanConf + .createWithDefault(false) val COMET_COLUMNAR_SHUFFLE_ASYNC_THREAD_NUM: ConfigEntry[Int] = conf("spark.comet.columnar.shuffle.async.thread.num") - .doc("Number of threads used for Comet async columnar shuffle per shuffle task. " + - "By default, this config is 3. 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.") + .doc( + "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.") .intConf .createWithDefault(3) val COMET_COLUMNAR_SHUFFLE_ASYNC_MAX_THREAD_NUM: ConfigEntry[Int] = { conf("spark.comet.columnar.shuffle.async.max.thread.num") .doc("Maximum number of threads on an executor used for Comet async columnar shuffle. " + - "By default, this config is 100. This is the upper bound of total number of 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 " + @@ -317,8 +315,7 @@ object CometConf extends ShimCometConf { "Higher value means more memory requirement to buffer shuffle data before " + "flushing to disk. As Comet uses columnar shuffle which is columnar format, " + "higher value usually helps to improve shuffle data compression ratio. This is " + - "internal config for testing purpose or advanced tuning. By default, " + - "this config is Int.Max.") + "internal config for testing purpose or advanced tuning.") .internal() .intConf .createWithDefault(Int.MaxValue) @@ -341,8 +338,7 @@ object CometConf extends ShimCometConf { .doc( "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`. " + - "By default, this config is 1.0.") + "calculated by `spark.comet.memory.overhead.factor` * `spark.executor.memory`.") .doubleConf .checkValue( factor => factor > 0, @@ -360,11 +356,12 @@ object CometConf extends ShimCometConf { val COMET_SHUFFLE_PREFER_DICTIONARY_RATIO: ConfigEntry[Double] = conf( "spark.comet.shuffle.preferDictionary.ratio") - .doc("The ratio of total values to distinct values in a string column to decide whether to " + - "prefer dictionary encoding when shuffling the column. If the ratio is higher than " + - "this config, dictionary encoding will be used on shuffling string column. This config " + - "is effective if it is higher than 1.0. By default, this config is 10.0. Note that this " + - "config is only used when `spark.comet.exec.shuffle.mode` is `jvm`.") + .doc( + "The ratio of total values to distinct values in a string column to decide whether to " + + "prefer dictionary encoding when shuffling the column. If the ratio is higher than " + + "this config, dictionary encoding will be used on shuffling string column. This config " + + "is effective if it is higher than 1.0. Note that this " + + "config is only used when `spark.comet.exec.shuffle.mode` is `jvm`.") .doubleConf .createWithDefault(10.0) @@ -377,7 +374,7 @@ object CometConf extends ShimCometConf { val COMET_DEBUG_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.debug.enabled") .doc( - "Whether to enable debug mode for Comet. By default, this config is false. " + + "Whether to enable debug mode for Comet. " + "When enabled, Comet will do additional checks for debugging purpose. For example, " + "validating array when importing arrays from JVM at native side. Note that these " + "checks may be expensive in performance and should only be enabled for debugging " + @@ -437,19 +434,19 @@ object CometConf extends ShimCometConf { "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. Default value is 0.7.") + "acquisition.") .doubleConf .createWithDefault(0.7) - val COMET_PARQUET_ENABLE_DIRECT_BUFFER: ConfigEntry[Boolean] = conf( - "spark.comet.parquet.enable.directBuffer") - .doc("Whether to use Java direct byte buffer when reading Parquet. By default, this is false") - .booleanConf - .createWithDefault(false) + val COMET_PARQUET_ENABLE_DIRECT_BUFFER: ConfigEntry[Boolean] = + conf("spark.comet.parquet.enable.directBuffer") + .doc("Whether to use Java direct byte buffer when reading Parquet.") + .booleanConf + .createWithDefault(false) val COMET_SCAN_PREFETCH_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.preFetch.enabled") - .doc("Whether to enable pre-fetching feature of CometScan. By default is disabled.") + .doc("Whether to enable pre-fetching feature of CometScan.") .booleanConf .createWithDefault(false) @@ -457,7 +454,7 @@ object CometConf extends ShimCometConf { conf("spark.comet.scan.preFetch.threadNum") .doc( "The number of threads running pre-fetching for CometScan. Effective if " + - s"${COMET_SCAN_PREFETCH_ENABLED.key} is enabled. By default it is 2. Note that more " + + s"${COMET_SCAN_PREFETCH_ENABLED.key} is enabled. Note that more " + "pre-fetching threads means more memory requirement to store pre-fetched row groups.") .intConf .createWithDefault(2) diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index 031322e9f..10459d4d2 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -123,6 +123,7 @@ The following cast operations are generally compatible with Spark except for the | decimal | long | | | decimal | float | | | decimal | double | | +| decimal | string | There can be formatting differences in some case due to Spark using scientific notation where Comet does not | | string | boolean | | | string | byte | | | string | short | | diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 1618a0345..dc97e3c59 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -26,14 +26,14 @@ Comet provides the following configuration settings. | spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | | spark.comet.caseConversion.enabled | Java uses locale-specific rules when converting strings to upper or lower case and Rust does not, so we disable upper and lower by default. | false | | spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | false | -| spark.comet.columnar.shuffle.async.enabled | Whether to enable asynchronous shuffle for Arrow-based shuffle. By default, this config is false. | false | -| spark.comet.columnar.shuffle.async.max.thread.num | Maximum number of threads on an executor used for Comet async columnar shuffle. By default, this config is 100. 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. By default, this config is 3. 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`. By default, this config is 1.0. | 1.0 | +| 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 | -| spark.comet.debug.enabled | Whether to enable debug mode for Comet. By default, this config is false. When enabled, Comet will do additional checks for debugging purpose. For example, validating array when importing arrays from JVM at native side. Note that these checks may be expensive in performance and should only be enabled for debugging purpose. | false | +| spark.comet.debug.enabled | Whether to enable debug mode for Comet. When enabled, Comet will do additional checks for debugging purpose. For example, validating array when importing arrays from JVM at native side. Note that these checks may be expensive in performance and should only be enabled for debugging purpose. | false | | spark.comet.dppFallback.enabled | Whether to fall back to Spark for queries that use DPP. | true | | spark.comet.enabled | Whether to enable Comet extension for Spark. When this is turned on, Spark will use Comet to read Parquet data source. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this config is the value of the env var `ENABLE_COMET` if set, or true otherwise. | true | | spark.comet.exceptionOnDatetimeRebase | Whether to throw exception when seeing dates/timestamps from the legacy hybrid (Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according to the Proleptic Gregorian calendar. When this is true, Comet will throw exceptions when seeing these dates/timestamps that were written by Spark version before 3.0. If this is false, these dates/timestamps will be read as if they were written to the Proleptic Gregorian calendar and will not be rebased. | false | @@ -42,13 +42,13 @@ Comet provides the following configuration settings. | spark.comet.exec.broadcastHashJoin.enabled | Whether to enable broadcastHashJoin by default. | true | | spark.comet.exec.coalesce.enabled | Whether to enable coalesce by default. | true | | spark.comet.exec.collectLimit.enabled | Whether to enable collectLimit by default. | true | -| spark.comet.exec.enabled | Whether to enable Comet native vectorized execution for Spark. This controls whether Spark should convert operators into their Comet counterparts and execute them in native space. Note: each operator is associated with a separate config in the format of 'spark.comet.exec..enabled' at the moment, and both the config and this need to be turned on, in order for the operator to be executed in native. By default, this config is true. | true | +| spark.comet.exec.enabled | Whether to enable Comet native vectorized execution for Spark. This controls whether Spark should convert operators into their Comet counterparts and execute them in native space. Note: each operator is associated with a separate config in the format of 'spark.comet.exec..enabled' at the moment, and both the config and this need to be turned on, in order for the operator to be executed in native. | true | | spark.comet.exec.expand.enabled | Whether to enable expand by default. | true | | spark.comet.exec.filter.enabled | Whether to enable filter by default. | true | | spark.comet.exec.globalLimit.enabled | Whether to enable globalLimit by default. | true | | spark.comet.exec.hashJoin.enabled | Whether to enable hashJoin by default. | true | | spark.comet.exec.localLimit.enabled | Whether to enable localLimit by default. | true | -| 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. Default value is 0.7. | 0.7 | +| 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 | @@ -63,18 +63,18 @@ Comet provides the following configuration settings. | spark.comet.explain.native.enabled | When this setting is enabled, Comet will provide a tree representation of the native query plan before execution and again after execution, with metrics. | false | | spark.comet.explain.verbose.enabled | When this setting is enabled, Comet will provide a verbose tree representation of the extended information. | false | | spark.comet.explainFallback.enabled | When this setting is enabled, Comet will provide logging explaining the reason(s) why a query stage cannot be executed natively. Set this to false to reduce the amount of logging. | false | -| spark.comet.memory.overhead.factor | Fraction of executor memory to be allocated as additional non-heap memory per executor process for Comet. Default value is 0.2. | 0.2 | +| spark.comet.memory.overhead.factor | Fraction of executor memory to be allocated as additional non-heap memory per executor process for Comet. | 0.2 | | spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB. | 402653184b | | spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | -| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. By default, this is false | false | -| spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. The option is off by default. | false | -| spark.comet.parquet.read.io.mergeRanges | When enabled the parallel reader will try to merge ranges of data that are separated by less than 'comet.parquet.read.io.mergeRanges.delta' bytes. Longer continuous reads are faster on cloud storage. The default behavior is to merge consecutive ranges. | true | +| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. | false | +| spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. | false | +| spark.comet.parquet.read.io.mergeRanges | When enabled the parallel reader will try to merge ranges of data that are separated by less than 'comet.parquet.read.io.mergeRanges.delta' bytes. Longer continuous reads are faster on cloud storage. | true | | spark.comet.parquet.read.io.mergeRanges.delta | The delta in bytes between consecutive read ranges below which the parallel reader will try to merge the ranges. The default is 8MB. | 8388608 | -| spark.comet.parquet.read.parallel.io.enabled | Whether to enable Comet's parallel reader for Parquet files. The parallel reader reads ranges of consecutive data in a file in parallel. It is faster for large files and row groups but uses more resources. The parallel reader is enabled by default. | true | +| spark.comet.parquet.read.parallel.io.enabled | Whether to enable Comet's parallel reader for Parquet files. The parallel reader reads ranges of consecutive data in a file in parallel. It is faster for large files and row groups but uses more resources. | true | | spark.comet.parquet.read.parallel.io.thread-pool.size | The maximum number of parallel threads the parallel reader will use in a single executor. For executors configured with a smaller number of cores, use a smaller number. | 16 | | spark.comet.regexp.allowIncompatible | Comet is not currently fully compatible with Spark for all regular expressions. Set this config to true to allow them anyway using Rust's regular expression engine. See compatibility guide for more information. | false | -| spark.comet.scan.enabled | Whether to enable native scans. When this is turned on, Spark will use Comet to read supported data sources (currently only Parquet is supported natively). Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this config is true. | true | -| spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. By default is disabled. | false | -| spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. By default it is 2. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | -| spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. By default, this config is 10.0. Note that this config is only used when `spark.comet.exec.shuffle.mode` is `jvm`. | 10.0 | +| spark.comet.scan.enabled | Whether to enable native scans. When this is turned on, Spark will use Comet to read supported data sources (currently only Parquet is supported natively). Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | true | +| spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. | false | +| spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | +| spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. Note that this config is only used when `spark.comet.exec.shuffle.mode` is `jvm`. | 10.0 | | spark.comet.sparkToColumnar.supportedOperatorList | A comma-separated list of operators that will be converted to Arrow columnar format when 'spark.comet.sparkToColumnar.enabled' is true | Range,InMemoryTableScan | diff --git a/docs/source/user-guide/kubernetes.md b/docs/source/user-guide/kubernetes.md index c69c1707a..cd923db88 100644 --- a/docs/source/user-guide/kubernetes.md +++ b/docs/source/user-guide/kubernetes.md @@ -21,8 +21,8 @@ ## Comet Docker Images -Run the following command from the root of this repository to build the Comet Docker image, or use a published -Docker image from https://github.com/orgs/apache/packages?repo_name=datafusion-comet +Run the following command from the root of this repository to build the Comet Docker image, or use a [published +Docker image](https://github.com/orgs/apache/packages?repo_name=datafusion-comet) ```shell docker build -t apache/datafusion-comet -f kube/Dockerfile . @@ -78,7 +78,7 @@ spec: "spark.comet.cast.allowIncompatible": "true" "spark.comet.exec.shuffle.enabled": "true" "spark.comet.exec.shuffle.mode": "auto" - "conf spark.shuffle.manager": "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" + "spark.shuffle.manager": "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" sparkVersion: 3.4.3 driver: labels: @@ -106,4 +106,4 @@ Check application status ```bash kubectl describe sparkapplication --namespace=spark-operator ``` -More info on Kube Spark operator https://www.kubeflow.org/docs/components/spark-operator/getting-started/ \ No newline at end of file +More info on [Kube Spark operator](https://www.kubeflow.org/docs/components/spark-operator/getting-started/) \ No newline at end of file diff --git a/native/Cargo.lock b/native/Cargo.lock index 955958483..c3a664ff3 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "addr2line" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5fb1d8e4442bd405fdfd1dacb42792696b0cf9cb15882e5d097b742a676d375" +checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" dependencies = [ "gimli", ] @@ -57,9 +57,9 @@ dependencies = [ [[package]] name = "allocator-api2" -version = "0.2.18" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" +checksum = "45862d1c77f2228b9e10bc609d5bc203d86ebc9b87ad8d5d5167a6c9abf739d9" [[package]] name = "android-tzdata" @@ -84,15 +84,15 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstyle" -version = "1.0.8" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bec1de6f59aedf83baf9ff929c98f2ad654b97c9510f4e70cf6f661d49fd5b1" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anyhow" -version = "1.0.89" +version = "1.0.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86fdf8605db99b54d3cd748a44c6d04df638eb5dafb219b135d0149bd0db01f6" +checksum = "4c95c10ba0b00a02636238b814946408b1322d5ac4760326e6fb8ec956d85775" [[package]] name = "arc-swap" @@ -114,9 +114,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9ba0d7248932f4e2a12fb37f0a2e3ec82b3bdedbac2a1dce186e036843b8f8c" +checksum = "4caf25cdc4a985f91df42ed9e9308e1adbcd341a31a72605c697033fcef163e3" dependencies = [ "arrow-arith", "arrow-array", @@ -135,9 +135,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d60afcdc004841a5c8d8da4f4fa22d64eb19c0c01ef4bcedd77f175a7cf6e38f" +checksum = "91f2dfd1a7ec0aca967dfaa616096aec49779adc8eccec005e2f5e4111b1192a" dependencies = [ "arrow-array", "arrow-buffer", @@ -150,9 +150,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f16835e8599dbbb1659fd869d865254c4cf32c6c2bb60b6942ac9fc36bfa5da" +checksum = "d39387ca628be747394890a6e47f138ceac1aa912eab64f02519fed24b637af8" dependencies = [ "ahash", "arrow-buffer", @@ -161,15 +161,15 @@ dependencies = [ "chrono", "chrono-tz 0.10.0", "half", - "hashbrown", + "hashbrown 0.14.5", "num", ] [[package]] name = "arrow-buffer" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a1f34f0faae77da6b142db61deba2cb6d60167592b178be317b341440acba80" +checksum = "9e51e05228852ffe3eb391ce7178a0f97d2cf80cc6ef91d3c4a6b3cb688049ec" dependencies = [ "bytes", "half", @@ -178,9 +178,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "450e4abb5775bca0740bec0bcf1b1a5ae07eff43bd625661c4436d8e8e4540c4" +checksum = "d09aea56ec9fa267f3f3f6cdab67d8a9974cbba90b3aa38c8fe9d0bb071bd8c1" dependencies = [ "arrow-array", "arrow-buffer", @@ -199,9 +199,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3a4e4d63830a341713e35d9a42452fbc6241d5f42fa5cf6a4681b8ad91370c4" +checksum = "c07b5232be87d115fde73e32f2ca7f1b353bff1b44ac422d3c6fc6ae38f11f0d" dependencies = [ "arrow-array", "arrow-buffer", @@ -218,9 +218,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b1e618bbf714c7a9e8d97203c806734f012ff71ae3adc8ad1b075689f540634" +checksum = "b98ae0af50890b494cebd7d6b04b35e896205c1d1df7b29a6272c5d0d0249ef5" dependencies = [ "arrow-buffer", "arrow-schema", @@ -230,9 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f98e983549259a2b97049af7edfb8f28b8911682040e99a94e4ceb1196bd65c2" +checksum = "0ed91bdeaff5a1c00d28d8f73466bcb64d32bbd7093b5a30156b4b9f4dba3eee" dependencies = [ "arrow-array", "arrow-buffer", @@ -245,9 +245,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b198b9c6fcf086501730efbbcb483317b39330a116125af7bb06467d04b352a3" +checksum = "0471f51260a5309307e5d409c9dc70aede1cd9cf1d4ff0f0a1e8e1a2dd0e0d3c" dependencies = [ "arrow-array", "arrow-buffer", @@ -265,9 +265,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2427f37b4459a4b9e533045abe87a5183a5e0995a3fc2c2fd45027ae2cc4ef3f" +checksum = "2883d7035e0b600fb4c30ce1e50e66e53d8656aa729f2bfa4b51d359cf3ded52" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,9 +280,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15959657d92e2261a7a323517640af87f5afd9fd8a6492e424ebee2203c567f6" +checksum = "552907e8e587a6fde4f8843fd7a27a576a260f65dab6c065741ea79f633fc5be" dependencies = [ "ahash", "arrow-array", @@ -294,18 +294,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbf0388a18fd7f7f3fe3de01852d30f54ed5182f9004db700fbe3ba843ed2794" +checksum = "539ada65246b949bd99ffa0881a9a15a4a529448af1a07a9838dd78617dafab1" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "arrow-select" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b83e5723d307a38bf00ecd2972cd078d1339c7fd3eb044f609958a9a24463f3a" +checksum = "6259e566b752da6dceab91766ed8b2e67bf6270eb9ad8a6e07a33c1bede2b125" dependencies = [ "ahash", "arrow-array", @@ -317,9 +317,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ab3db7c09dd826e74079661d84ed01ed06547cf75d52c2818ef776d0d852305" +checksum = "f3179ccbd18ebf04277a095ba7321b93fd1f774f18816bd5f6b3ce2f594edb6c" dependencies = [ "arrow-array", "arrow-buffer", @@ -340,13 +340,13 @@ checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" [[package]] name = "async-trait" -version = "0.1.82" +version = "0.1.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" +checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -360,9 +360,9 @@ dependencies = [ [[package]] name = "autocfg" -version = "1.3.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" +checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "backtrace" @@ -441,9 +441,9 @@ dependencies = [ [[package]] name = "brotli" -version = "6.0.0" +version = "7.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74f7971dbd9326d58187408ab83117d8ac1bb9c17b085fdacd1cf2f598719b6b" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -478,9 +478,9 @@ checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytemuck" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94bbb0ad554ad961ddc5da507a12a29b14e4ae5bda06b19f575a3e6079d2e2ae" +checksum = "8334215b81e418a0a7bdb8ef0849474f40bb10c8b71f1c4ed315cff49f32494d" [[package]] name = "byteorder" @@ -490,9 +490,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.7.1" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8318a53db07bb3f8dca91a600466bdb3f2eaadeedfdbcf02e1accbad9271ba50" +checksum = "9ac0150caa2ae65ca5bd83f25c7de183dea78d4d366469f148435e2acfbad0da" [[package]] name = "cast" @@ -502,9 +502,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.1.20" +version = "1.1.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45bcde016d64c21da4be18b655631e5ab6d3107607e71a73a9f53eb48aae23fb" +checksum = "40545c26d092346d8a8dab71ee48e7685a7a9cba76e634790c215b41a4a7b4cf" dependencies = [ "jobserver", "libc", @@ -607,18 +607,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.17" +version = "4.5.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e5a21b8495e732f1b3c364c9949b201ca7bae518c502c80256c96ad79eaf6ac" +checksum = "b97f376d85a664d5837dbae44bf546e6477a679ff6610010f17276f686d867e8" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.17" +version = "4.5.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cf2dd12af7a047ad9d6da2b6b249759a22a7abc0f474c1dae1777afa4b21a73" +checksum = "19bc80abd44e4bed93ca373a0704ccbd1b710dc5749406201bb018272808dc54" dependencies = [ "anstyle", "clap_lex", @@ -694,9 +694,9 @@ dependencies = [ [[package]] name = "cpufeatures" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "608697df725056feaccfa42cffdaeeec3fccc4ffc38358ecd19b243e716a78e0" +checksum = "0ca741a962e1b0bff6d724a1a0958b686406e853bb14061f218562e1896f95e6" dependencies = [ "libc", ] @@ -791,9 +791,9 @@ dependencies = [ [[package]] name = "csv" -version = "1.3.0" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" +checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" dependencies = [ "csv-core", "itoa", @@ -818,7 +818,7 @@ checksum = "5041cc499144891f3790297212f32a74fb938e5136a14943f338ef9e0ae276cf" dependencies = [ "cfg-if", "crossbeam-utils", - "hashbrown", + "hashbrown 0.14.5", "lock_api", "once_cell", "parking_lot_core", @@ -826,8 +826,9 @@ dependencies = [ [[package]] name = "datafusion" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cbba0799cf6913b456ed07a94f0f3b6e12c62a5d88b10809e2284a0f2b915c05" dependencies = [ "ahash", "arrow", @@ -855,7 +856,7 @@ dependencies = [ "futures", "glob", "half", - "hashbrown", + "hashbrown 0.14.5", "indexmap", "itertools 0.13.0", "log", @@ -875,8 +876,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7493c5c2d40eec435b13d92e5703554f4efc7059451fcb8d3a79580ff0e45560" dependencies = [ "arrow-schema", "async-trait", @@ -968,13 +970,14 @@ dependencies = [ "rand", "regex", "thiserror", - "twox-hash 2.0.0", + "twox-hash 2.0.1", ] [[package]] name = "datafusion-common" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24953049ebbd6f8964f91f60aa3514e121b5e81e068e33b60e77815ab369b25c" dependencies = [ "ahash", "arrow", @@ -983,7 +986,8 @@ dependencies = [ "arrow-schema", "chrono", "half", - "hashbrown", + "hashbrown 0.14.5", + "indexmap", "instant", "libc", "num_cpus", @@ -996,8 +1000,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f06df4ef76872e11c924d3c814fd2a8dd09905ed2e2195f71c857d78abd19685" dependencies = [ "log", "tokio", @@ -1005,8 +1010,9 @@ dependencies = [ [[package]] name = "datafusion-execution" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bbdcb628d690f3ce5fea7de81642b514486d58ff9779a51f180a69a4eadb361" dependencies = [ "arrow", "chrono", @@ -1014,7 +1020,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown", + "hashbrown 0.14.5", "log", "object_store", "parking_lot", @@ -1025,8 +1031,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8036495980e3131f706b7d33ab00b4492d73dc714e3cb74d11b50f9602a73246" dependencies = [ "ahash", "arrow", @@ -1048,18 +1055,21 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4da0f3cb4669f9523b403d6b5a0ec85023e0ab3bf0183afd1517475b3e64fdd2" dependencies = [ "arrow", "datafusion-common", + "itertools 0.13.0", "paste", ] [[package]] name = "datafusion-functions" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f52c4012648b34853e40a2c6bcaa8772f837831019b68aca384fb38436dba162" dependencies = [ "arrow", "arrow-buffer", @@ -1070,7 +1080,7 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "hashbrown", + "hashbrown 0.14.5", "hex", "itertools 0.13.0", "log", @@ -1084,8 +1094,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5b8bb624597ba28ed7446df4a9bd7c7a7bde7c578b6b527da3f47371d5f6741" dependencies = [ "ahash", "arrow", @@ -1104,8 +1115,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fb06208fc470bc8cf1ce2d9a1159d42db591f2c7264a8c1776b53ad8f675143" dependencies = [ "ahash", "arrow", @@ -1117,8 +1129,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fca25bbb87323716d05e54114666e942172ccca23c5a507e9c7851db6e965317" dependencies = [ "arrow", "arrow-array", @@ -1139,12 +1152,14 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ae23356c634e54c59f7c51acb7a5b9f6240ffb2cf997049a1a24a8a88598dbe" dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-functions-window-common", + "datafusion-physical-expr", "datafusion-physical-expr-common", "log", "paste", @@ -1152,8 +1167,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4b3d6ff7794acea026de36007077a06b18b89e4f9c3fea7f2215f9f7dd9059b" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1161,8 +1177,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bec6241eb80c595fa0e1a8a6b69686b5cf3bd5fdacb8319582a0943b0bd788aa" dependencies = [ "arrow", "async-trait", @@ -1170,7 +1187,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown", + "hashbrown 0.14.5", "indexmap", "itertools 0.13.0", "log", @@ -1180,8 +1197,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3370357b8fc75ec38577700644e5d1b0bc78f38babab99c0b8bd26bafb3e4335" dependencies = [ "ahash", "arrow", @@ -1197,7 +1215,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-physical-expr-common", "half", - "hashbrown", + "hashbrown 0.14.5", "indexmap", "itertools 0.13.0", "log", @@ -1207,21 +1225,23 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8b7734d94bf2fa6f6e570935b0ddddd8421179ce200065be97874e13d46a47b" dependencies = [ "ahash", "arrow", "datafusion-common", "datafusion-expr-common", - "hashbrown", + "hashbrown 0.14.5", "rand", ] [[package]] name = "datafusion-physical-optimizer" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7eee8c479522df21d7b395640dff88c5ed05361852dce6544d7c98e9dbcebffe" dependencies = [ "arrow", "arrow-schema", @@ -1235,8 +1255,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "17e1fc2e2c239d14e8556f2622b19a726bf6bc6962cc00c71fc52626274bee24" dependencies = [ "ahash", "arrow", @@ -1256,7 +1277,7 @@ dependencies = [ "datafusion-physical-expr-common", "futures", "half", - "hashbrown", + "hashbrown 0.14.5", "indexmap", "itertools 0.13.0", "log", @@ -1269,14 +1290,16 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "42.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3b6aac2#3b6aac2fcecdb003427f9475f061ed2cc52e8558" +version = "43.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63e3a4ed41dbee20a5d947a59ca035c225d67dc9cbe869c10f66dcdf25e7ce51" dependencies = [ "arrow", "arrow-array", "arrow-schema", "datafusion-common", "datafusion-expr", + "indexmap", "log", "regex", "sqlparser", @@ -1320,6 +1343,17 @@ dependencies = [ "subtle", ] +[[package]] +name = "displaydoc" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.87", +] + [[package]] name = "either" version = "1.13.0" @@ -1344,9 +1378,9 @@ dependencies = [ [[package]] name = "fastrand" -version = "2.1.1" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8c02a5121d4ea3eb16a80748c74f5549a5665e4c21333c6098f283870fbdea6" +checksum = "486f806e73c5707928240ddc295403b1b93c96a02038563881c4a2fd84b81ac4" [[package]] name = "findshlibs" @@ -1378,9 +1412,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.33" +version = "1.0.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "324a1be68054ef05ad64b861cc9eaf1d623d2d8cb25b4bf2cb9cdd902b4bf253" +checksum = "a1b589b4dc103969ad3cf85c950899926ec64300a1a46d76c03a6072957036f0" dependencies = [ "crc32fast", "miniz_oxide", @@ -1403,9 +1437,9 @@ dependencies = [ [[package]] name = "futures" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" dependencies = [ "futures-channel", "futures-core", @@ -1418,9 +1452,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" dependencies = [ "futures-core", "futures-sink", @@ -1428,15 +1462,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" +checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" [[package]] name = "futures-executor" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" dependencies = [ "futures-core", "futures-task", @@ -1445,38 +1479,38 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" +checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" [[package]] name = "futures-macro" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] name = "futures-sink" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" +checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" [[package]] name = "futures-task" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" +checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" [[package]] name = "futures-util" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" dependencies = [ "futures-channel", "futures-core", @@ -1513,9 +1547,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32085ea23f3234fc7846555e85283ba4de91e21016dc0455a16286d87a292d64" +checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" @@ -1544,6 +1578,12 @@ dependencies = [ "allocator-api2", ] +[[package]] +name = "hashbrown" +version = "0.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a9bfc1af68b1726ea47d3d5109de126281def866b33970e10fbab11b5dafab3" + [[package]] name = "heck" version = "0.3.3" @@ -1615,24 +1655,153 @@ dependencies = [ "cc", ] +[[package]] +name = "icu_collections" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db2fa452206ebee18c4b5c2274dbf1de17008e874b4dc4f0aea9d01ca79e4526" +dependencies = [ + "displaydoc", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locid" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13acbb8371917fc971be86fc8057c41a64b521c184808a698c02acc242dbf637" +dependencies = [ + "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_locid_transform" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01d11ac35de8e40fdeda00d9e1e9d92525f3f9d887cdd7aa81d727596788b54e" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_locid_transform_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_locid_transform_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdc8ff3388f852bede6b579ad4e978ab004f139284d7b28715f773507b946f6e" + +[[package]] +name = "icu_normalizer" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19ce3e0da2ec68599d193c93d088142efd7f9c5d6fc9b803774855747dc6a84f" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_normalizer_data", + "icu_properties", + "icu_provider", + "smallvec", + "utf16_iter", + "utf8_iter", + "write16", + "zerovec", +] + +[[package]] +name = "icu_normalizer_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8cafbf7aa791e9b22bec55a167906f9e1215fd475cd22adfcf660e03e989516" + +[[package]] +name = "icu_properties" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93d6020766cfc6302c15dbbc9c8778c37e62c14427cb7f6e601d849e092aeef5" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_locid_transform", + "icu_properties_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_properties_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67a8effbc3dd3e4ba1afa8ad918d5684b8868b3b26500753effea8d2eed19569" + +[[package]] +name = "icu_provider" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ed421c8a8ef78d3e2dbc98a973be2f3770cb42b606e3ab18d6237c4dfde68d9" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_provider_macros", + "stable_deref_trait", + "tinystr", + "writeable", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_provider_macros" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.87", +] + [[package]] name = "idna" -version = "0.5.0" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "686f825264d630750a544639377bae737628043f20d38bbc029e8f29ea968a7e" +dependencies = [ + "idna_adapter", + "smallvec", + "utf8_iter", +] + +[[package]] +name = "idna_adapter" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" +checksum = "daca1df1c957320b2cf139ac61e7bd64fed304c5040df000a745aa1de3b4ef71" dependencies = [ - "unicode-bidi", - "unicode-normalization", + "icu_normalizer", + "icu_properties", ] [[package]] name = "indexmap" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" +checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" dependencies = [ "equivalent", - "hashbrown", + "hashbrown 0.15.1", ] [[package]] @@ -1726,12 +1895,11 @@ checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" [[package]] name = "java-locator" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2abecabd9961c5e01405a6426687fcf1bd94a269927137e4c3cc1a7419b93fd" +checksum = "6f25f28894af6a5dd349ed5ec46e178654e75f62edb6717ac74007102a57deb5" dependencies = [ "glob", - "lazy_static", ] [[package]] @@ -1769,9 +1937,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.70" +version = "0.3.72" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1868808506b929d7b0cfa8f75951347aa71bb21144b7791bae35d9bccfcfe37a" +checksum = "6a88f1bda2bd75b0452a14784937d796722fdebfe50df998aeb3f0b7603019a9" dependencies = [ "wasm-bindgen", ] @@ -1848,9 +2016,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.158" +version = "0.2.162" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8adc4bb1803a324070e64a98ae98f38934d91957a99cfb3a43dcbc01bc56439" +checksum = "18d287de67fe55fd7e1581fe933d965a5a9477b38e949cfa9f8574ef01506398" [[package]] name = "libloading" @@ -1864,9 +2032,9 @@ dependencies = [ [[package]] name = "libm" -version = "0.2.8" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" +checksum = "8355be11b20d696c8f18f6cc018c4e372165b1fa8126cef092399c9951984ffa" [[package]] name = "libmimalloc-sys" @@ -1884,6 +2052,12 @@ version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" +[[package]] +name = "litemap" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "643cb0b8d4fcc284004d5fd0d67ccf61dfffadb7f75e1e71bc420f4688a3a704" + [[package]] name = "lock_api" version = "0.4.12" @@ -1939,18 +2113,18 @@ dependencies = [ [[package]] name = "lz4" -version = "1.27.0" +version = "1.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a231296ca742e418c43660cb68e082486ff2538e8db432bc818580f3965025ed" +checksum = "4d1febb2b4a79ddd1980eede06a8f7902197960aa0383ffcfdd62fe723036725" dependencies = [ "lz4-sys", ] [[package]] name = "lz4-sys" -version = "1.11.0" +version = "1.11.1+lz4-1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb44a01837a858d47e5a630d2ccf304c8efcc4b83b8f9f75b7a9ee4fcc6e57d" +checksum = "6bd8c0d6c6ed0cd30b3652886bb8711dc4bb01d637a68105a3d5158039b418e6" dependencies = [ "cc", "libc", @@ -2121,18 +2295,18 @@ dependencies = [ [[package]] name = "object" -version = "0.36.4" +version = "0.36.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "084f1a5821ac4c651660a94a7153d27ac9d8a53736203f58b31945ded098070a" +checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25a0c4b3a0e31f8b66f71ad8064521efa773910196e2cde791436f13409f3b45" +checksum = "6eb4c22c6154a1e759d7099f9ffad7cc5ef8245f9efbab4a41b92623079c82f3" dependencies = [ "async-trait", "bytes", @@ -2151,9 +2325,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.19.0" +version = "1.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" +checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" [[package]] name = "oorandom" @@ -2195,9 +2369,9 @@ dependencies = [ [[package]] name = "parquet" -version = "53.1.0" +version = "53.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "310c46a70a3ba90d98fec39fa2da6d9d731e544191da6fb56c9d199484d0dd3e" +checksum = "dea02606ba6f5e856561d8d507dba8bac060aefca2a6c0f1aa1d361fed91ff3e" dependencies = [ "ahash", "arrow-array", @@ -2208,13 +2382,13 @@ dependencies = [ "arrow-schema", "arrow-select", "base64", - "brotli 6.0.0", + "brotli 7.0.0", "bytes", "chrono", "flate2", "futures", "half", - "hashbrown", + "hashbrown 0.14.5", "lz4_flex", "num", "num-bigint", @@ -2300,9 +2474,9 @@ dependencies = [ [[package]] name = "pin-project-lite" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" +checksum = "915a1e146535de9163f3987b8944ed8cf49a18bb0056bcebcdcece385cece4ff" [[package]] name = "pin-utils" @@ -2312,9 +2486,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkg-config" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" +checksum = "953ec861398dccce10c670dfeaf3ec4911ca479e9c02154b3a215178c5f566f2" [[package]] name = "plotters" @@ -2376,9 +2550,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.86" +version = "1.0.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77" +checksum = "f139b0662de085916d1fb67d2b4169d1addddda1919e696f3252b740b629986e" dependencies = [ "unicode-ident", ] @@ -2446,7 +2620,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -2529,18 +2703,18 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.4" +version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0884ad60e090bf1345b93da0a5de8923c93884cd03f40dfcfddd3b4bee661853" +checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "regex" -version = "1.10.6" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4219d74c6b67a3654a9fbebc4b419e22126d13d2f3c4a07ee0cb61ff79a79619" +checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" dependencies = [ "aho-corasick", "memchr", @@ -2550,9 +2724,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.7" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df" +checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" dependencies = [ "aho-corasick", "memchr", @@ -2561,9 +2735,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" +checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" [[package]] name = "rgb" @@ -2591,9 +2765,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.37" +version = "0.38.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8acb788b847c24f28525660c4d7758620a7210875711f79e7f663cc152726811" +checksum = "99e4ea3e1cdc4b559b8e5650f9c8e5998e3e5c1343b4eaf034565f32318d63c0" dependencies = [ "bitflags 2.6.0", "errno", @@ -2604,9 +2778,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.17" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "955d28af4278de8121b7ebeb796b6a45735dc01436d898801014aced2773a3d6" +checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" [[package]] name = "ryu" @@ -2643,9 +2817,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.210" +version = "1.0.214" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8e3592472072e6e22e0a54d5904d9febf8508f65fb8552499a1abc7d1078c3a" +checksum = "f55c3193aca71c12ad7890f1785d2b73e1b9f63a0bbc353c08ef26fe03fc56b5" dependencies = [ "serde_derive", ] @@ -2662,20 +2836,20 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.210" +version = "1.0.214" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "243902eda00fad750862fc144cea25caca5e20d615af0a81bee94ca738f1df1f" +checksum = "de523f781f095e28fa605cdce0f8307e451cc0fd14e2eb4cd2e98a355b147766" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] name = "serde_json" -version = "1.0.128" +version = "1.0.132" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ff5456707a1de34e7e37f2a6fd3d3f808c318259cbd01ab6377795054b483d8" +checksum = "d726bfaff4b320266d395898905d0eba0345aae23b54aee3a737e260fd46db03" dependencies = [ "itoa", "memchr", @@ -2742,23 +2916,23 @@ checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" [[package]] name = "snafu" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b835cb902660db3415a672d862905e791e54d306c6e8189168c7f3d9ae1c79d" +checksum = "223891c85e2a29c3fe8fb900c1fae5e69c2e42415e3177752e8718475efa5019" dependencies = [ "snafu-derive", ] [[package]] name = "snafu-derive" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d1e02fca405f6280643174a50c942219f0bbf4dbf7d480f1dd864d6f211ae5" +checksum = "03c3c6b7927ffe7ecaa769ee0e3994da3b8cafc8f444578982c83ecb161af917" dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -2785,7 +2959,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -2825,7 +2999,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -2836,9 +3010,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbolic-common" -version = "12.11.1" +version = "12.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fdf97c441f18a4f92425b896a4ec7a27e03631a0b1047ec4e34e9916a9a167e" +checksum = "3d4d73159efebfb389d819fd479afb2dbd57dcb3e3f4b7fcfa0e675f5a46c1cb" dependencies = [ "debugid", "memmap2", @@ -2848,9 +3022,9 @@ dependencies = [ [[package]] name = "symbolic-demangle" -version = "12.11.1" +version = "12.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc8ece6b129e97e53d1fbb3f61d33a6a9e5369b11d01228c068094d6d134eaea" +checksum = "a767859f6549c665011970874c3f541838b4835d5aaaa493d3ee383918be9f10" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -2870,20 +3044,31 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.77" +version = "2.0.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f35bcdf61fd8e7be6caf75f429fdca8beb3ed76584befb503b1569faee373ed" +checksum = "25aa4ce346d03a6dcd68dd8b4010bcb74e54e62c90c573f394c46eae99aba32d" dependencies = [ "proc-macro2", "quote", "unicode-ident", ] +[[package]] +name = "synstructure" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.87", +] + [[package]] name = "tempfile" -version = "3.12.0" +version = "3.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04cbcdd0c794ebb0d4cf35e88edd2f7d2c4c3e9a5a6dab322839b321c6a87a64" +checksum = "28cce251fcbc87fac86a866eeb0d6c2d536fc16d06f184bb61aeae11aa4cee0c" dependencies = [ "cfg-if", "fastrand", @@ -2894,22 +3079,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.63" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0342370b38b6a11b6cc11d6a805569958d54cfa061a29969c3b5ce2ea405724" +checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.63" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" +checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -2943,35 +3128,30 @@ dependencies = [ ] [[package]] -name = "tinytemplate" -version = "1.2.1" +name = "tinystr" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +checksum = "9117f5d4db391c1cf6927e7bea3db74b9a1c1add8f7eda9ffd5364f40f57b82f" dependencies = [ - "serde", - "serde_json", + "displaydoc", + "zerovec", ] [[package]] -name = "tinyvec" -version = "1.8.0" +name = "tinytemplate" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" dependencies = [ - "tinyvec_macros", + "serde", + "serde_json", ] -[[package]] -name = "tinyvec_macros" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" - [[package]] name = "tokio" -version = "1.40.0" +version = "1.41.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2b070231665d27ad9ec9b8df639893f46727666c6767db40317fbe920a5d998" +checksum = "22cfb5bee7a6a52939ca9224d6ac897bb669134078daa8735560897f69de4d33" dependencies = [ "backtrace", "bytes", @@ -2987,7 +3167,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -3009,7 +3189,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", ] [[package]] @@ -3033,9 +3213,9 @@ dependencies = [ [[package]] name = "twox-hash" -version = "2.0.0" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d09466de2fbca05ea830e16e62943f26607ab2148fb72b642505541711d99ad2" +checksum = "a6db6856664807f43c17fbaf2718e2381ac1476a449aa104f5f64622defa1245" dependencies = [ "rand", ] @@ -3055,27 +3235,12 @@ version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" -[[package]] -name = "unicode-bidi" -version = "0.3.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" - [[package]] name = "unicode-ident" version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e91b56cd4cadaeb79bbf1a5645f6b4f8dc5bde8834ad5894a8db35fda9efa1fe" -[[package]] -name = "unicode-normalization" -version = "0.1.24" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5033c97c4262335cded6d6fc3e5c18ab755e1a3dc96376350f3d8e9f009ad956" -dependencies = [ - "tinyvec", -] - [[package]] name = "unicode-segmentation" version = "1.12.0" @@ -3084,9 +3249,9 @@ checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" [[package]] name = "unicode-width" -version = "0.1.13" +version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" +checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" [[package]] name = "unsafe-any-ors" @@ -3105,20 +3270,32 @@ checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" [[package]] name = "url" -version = "2.5.2" +version = "2.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22784dbdf76fdde8af1aeda5622b546b422b6fc585325248a2bf9f5e41e94d6c" +checksum = "8d157f1b96d14500ffdc1f10ba712e780825526c03d9a49b4d0324b0d9113ada" dependencies = [ "form_urlencoded", "idna", "percent-encoding", ] +[[package]] +name = "utf16_iter" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8232dd3cdaed5356e0f716d285e4b40b932ac434100fe9b7e0e8e935b9e6246" + +[[package]] +name = "utf8_iter" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" + [[package]] name = "uuid" -version = "1.10.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81dfa00651efa65069b0b6b651f4aaa31ba9e3c3ce0137aaad053604ee7e0314" +checksum = "f8c5f0a0af699448548ad1a2fbf920fb4bee257eae39953ba95cb84891a0446a" dependencies = [ "getrandom", ] @@ -3147,9 +3324,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.93" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a82edfc16a6c469f5f44dc7b571814045d60404b55a0ee849f9bcfa2e63dd9b5" +checksum = "128d1e363af62632b8eb57219c8fd7877144af57558fb2ef0368d0087bddeb2e" dependencies = [ "cfg-if", "once_cell", @@ -3158,24 +3335,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.93" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9de396da306523044d3302746f1208fa71d7532227f15e347e2d93e4145dd77b" +checksum = "cb6dd4d3ca0ddffd1dd1c9c04f94b868c37ff5fac97c30b97cff2d74fce3a358" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.93" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "585c4c91a46b072c92e908d99cb1dcdf95c5218eeb6f3bf1efa991ee7a68cccf" +checksum = "e79384be7f8f5a9dd5d7167216f022090cf1f9ec128e6e6a482a2cb5c5422c56" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3183,28 +3360,28 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.93" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" +checksum = "26c6ab57572f7a24a4985830b120de1594465e5d500f24afe89e16b4e833ef68" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.93" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c62a0a307cb4a311d3a07867860911ca130c3494e8c2719593806c08bc5d0484" +checksum = "65fc09f10666a9f147042251e0dda9c18f166ff7de300607007e96bdebc1068d" [[package]] name = "web-sys" -version = "0.3.70" +version = "0.3.72" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26fdeaafd9bd129f65e7c031593c24d62186301e0c72c8978fa1678be7d532c0" +checksum = "f6488b90108c040df0fe62fa815cbdee25124641df01814dd7282749234c6112" dependencies = [ "js-sys", "wasm-bindgen", @@ -3410,6 +3587,42 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" +[[package]] +name = "write16" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" + +[[package]] +name = "writeable" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" + +[[package]] +name = "yoke" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c5b1314b079b0930c31e3af543d8ee1757b1951ae1e1565ec704403a7240ca5" +dependencies = [ + "serde", + "stable_deref_trait", + "yoke-derive", + "zerofrom", +] + +[[package]] +name = "yoke-derive" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "28cc31741b18cb6f1d5ff12f5b7523e3d6eb0852bbbad19d73905511d9849b95" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.87", + "synstructure", +] + [[package]] name = "zerocopy" version = "0.7.35" @@ -3428,7 +3641,50 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.87", +] + +[[package]] +name = "zerofrom" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91ec111ce797d0e0784a1116d0ddcdbea84322cd79e5d5ad173daeba4f93ab55" +dependencies = [ + "zerofrom-derive", +] + +[[package]] +name = "zerofrom-derive" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ea7b4a3637ea8669cedf0f1fd5c286a17f3de97b8dd5a70a6c167a1730e63a5" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.87", + "synstructure", +] + +[[package]] +name = "zerovec" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa2b893d79df23bfb12d5461018d408ea19dfafe76c2c7ef6d4eba614f8ff079" +dependencies = [ + "yoke", + "zerofrom", + "zerovec-derive", +] + +[[package]] +name = "zerovec-derive" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.87", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index f9af172e6..4b89231c7 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -30,23 +30,23 @@ license = "Apache-2.0" edition = "2021" # Comet uses the same minimum Rust version as DataFusion -rust-version = "1.75" +rust-version = "1.79" [workspace.dependencies] -arrow = { version = "53.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { version = "53.1.0" } -arrow-buffer = { version = "53.1.0" } -arrow-data = { version = "53.1.0" } -arrow-schema = { version = "53.1.0" } -parquet = { version = "53.1.0", default-features = false, features = ["experimental"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2" } -datafusion = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2", default-features = false, features = ["unicode_expressions", "crypto_expressions", "parquet"] } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2", features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "3b6aac2", default-features = false } +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-common = { version = "43.0.0" } +datafusion = { version = "43.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "parquet"] } +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 } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.4.0" } datafusion-comet-proto = { path = "proto", version = "0.4.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/benches/aggregate.rs b/native/core/benches/aggregate.rs index 14425f76c..c6209406f 100644 --- a/native/core/benches/aggregate.rs +++ b/native/core/benches/aggregate.rs @@ -151,7 +151,7 @@ fn create_aggregate( AggregateExec::try_new( AggregateMode::Partial, PhysicalGroupBy::new_single(vec![(c0, "c0".to_string())]), - vec![aggr_expr], + vec![aggr_expr.into()], vec![None], // no filter expressions scan, Arc::clone(schema), diff --git a/native/core/benches/bloom_filter_agg.rs b/native/core/benches/bloom_filter_agg.rs index 90e3e3f64..af3eb919e 100644 --- a/native/core/benches/bloom_filter_agg.rs +++ b/native/core/benches/bloom_filter_agg.rs @@ -121,7 +121,7 @@ fn create_aggregate( AggregateExec::try_new( mode, PhysicalGroupBy::new_single(vec![]), - vec![aggr_expr], + vec![aggr_expr.into()], vec![None], scan, Arc::clone(schema), diff --git a/native/core/src/execution/datafusion/expressions/sum_decimal.rs b/native/core/src/execution/datafusion/expressions/sum_decimal.rs index a3ce96b67..d885ff90b 100644 --- a/native/core/src/execution/datafusion/expressions/sum_decimal.rs +++ b/native/core/src/execution/datafusion/expressions/sum_decimal.rs @@ -536,7 +536,7 @@ mod tests { let aggregate = Arc::new(AggregateExec::try_new( AggregateMode::Partial, PhysicalGroupBy::new_single(vec![(c0, "c0".to_string())]), - vec![aggr_expr], + vec![aggr_expr.into()], vec![None], // no filter expressions scan, Arc::clone(&schema), diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index ff492fa21..48a653add 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -116,6 +116,7 @@ use datafusion_expr::{ }; use datafusion_physical_expr::expressions::{Literal, StatsType}; use datafusion_physical_expr::window::WindowExpr; +use datafusion_physical_expr::LexOrdering; use itertools::Itertools; use jni::objects::GlobalRef; use num::{BigInt, ToPrimitive}; @@ -881,11 +882,12 @@ impl PhysicalPlanner { .collect(); let num_agg = agg.agg_exprs.len(); + let aggr_expr = agg_exprs?.into_iter().map(Arc::new).collect(); let aggregate = Arc::new( datafusion::physical_plan::aggregates::AggregateExec::try_new( mode, group_by, - agg_exprs?, + aggr_expr, vec![None; num_agg], // no filter expressions Arc::clone(&child), Arc::clone(&schema), @@ -943,7 +945,7 @@ impl PhysicalPlanner { Ok(( scans, - Arc::new(SortExec::new(exprs?, child).with_fetch(fetch)), + Arc::new(SortExec::new(LexOrdering::new(exprs?), child).with_fetch(fetch)), )) } OpStruct::Scan(scan) => { @@ -2508,7 +2510,7 @@ mod tests { }; let expr = spark_expression::Expr { - expr_struct: Some(Eq(Box::new(spark_expression::Equal { + expr_struct: Some(Eq(Box::new(spark_expression::BinaryExpr { left: Some(Box::new(left)), right: Some(Box::new(right)), }))), diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs index c523f843f..455f19929 100644 --- a/native/core/src/parquet/mod.rs +++ b/native/core/src/parquet/mod.rs @@ -27,7 +27,7 @@ use std::{boxed::Box, ptr::NonNull, sync::Arc}; use crate::errors::{try_unwrap_or_throw, CometError}; -use arrow::ffi::{FFI_ArrowArray, FFI_ArrowSchema}; +use arrow::ffi::FFI_ArrowArray; /// JNI exposed methods use jni::JNIEnv; @@ -52,7 +52,6 @@ const STR_CLASS_NAME: &str = "java/lang/String"; /// Parquet read context maintained across multiple JNI calls. struct Context { pub column_reader: ColumnReader, - pub arrays: Option<(Arc, Arc)>, last_data_page: Option, } @@ -110,7 +109,6 @@ pub extern "system" fn Java_org_apache_comet_parquet_Native_initColumnReader( use_decimal_128 != 0, use_legacy_date_timestamp != 0, ), - arrays: None, last_data_page: None, }; let res = Box::new(ctx); @@ -539,24 +537,15 @@ pub extern "system" fn Java_org_apache_comet_parquet_Native_currentBatch( e: JNIEnv, _jclass: JClass, handle: jlong, -) -> jlongArray { - try_unwrap_or_throw(&e, |env| { + array_addr: jlong, + schema_addr: jlong, +) { + try_unwrap_or_throw(&e, |_env| { let ctx = get_context(handle)?; let reader = &mut ctx.column_reader; let data = reader.current_batch(); - let (array, schema) = data.to_spark()?; - - unsafe { - let arrow_array = Arc::from_raw(array as *const FFI_ArrowArray); - let arrow_schema = Arc::from_raw(schema as *const FFI_ArrowSchema); - ctx.arrays = Some((arrow_array, arrow_schema)); - - let res = env.new_long_array(2)?; - let buf: [i64; 2] = [array, schema]; - env.set_long_array_region(&res, 0, &buf) - .expect("set long array region failed"); - Ok(res.into_raw()) - } + data.move_to_spark(array_addr, schema_addr) + .map_err(|e| e.into()) }) } diff --git a/native/proto/src/proto/expr.proto b/native/proto/src/proto/expr.proto index 796ca5be1..3a8193f4a 100644 --- a/native/proto/src/proto/expr.proto +++ b/native/proto/src/proto/expr.proto @@ -28,51 +28,51 @@ message Expr { oneof expr_struct { Literal literal = 2; BoundReference bound = 3; - Add add = 4; - Subtract subtract = 5; - Multiply multiply = 6; - Divide divide = 7; + MathExpr add = 4; + MathExpr subtract = 5; + MathExpr multiply = 6; + MathExpr divide = 7; Cast cast = 8; - Equal eq = 9; - NotEqual neq = 10; - GreaterThan gt = 11; - GreaterThanEqual gt_eq = 12; - LessThan lt = 13; - LessThanEqual lt_eq = 14; - IsNull is_null = 15; - IsNotNull is_not_null = 16; - And and = 17; - Or or = 18; + BinaryExpr eq = 9; + BinaryExpr neq = 10; + BinaryExpr gt = 11; + BinaryExpr gt_eq = 12; + BinaryExpr lt = 13; + BinaryExpr lt_eq = 14; + UnaryExpr is_null = 15; + UnaryExpr is_not_null = 16; + BinaryExpr and = 17; + BinaryExpr or = 18; SortOrder sort_order = 19; Substring substring = 20; - StringSpace string_space = 21; + UnaryExpr string_space = 21; Hour hour = 22; Minute minute = 23; Second second = 24; CheckOverflow check_overflow = 25; - Like like = 26; - StartsWith startsWith = 27; - EndsWith endsWith = 28; - Contains contains = 29; - RLike rlike = 30; + BinaryExpr like = 26; + BinaryExpr startsWith = 27; + BinaryExpr endsWith = 28; + BinaryExpr contains = 29; + BinaryExpr rlike = 30; ScalarFunc scalarFunc = 31; - EqualNullSafe eqNullSafe = 32; - NotEqualNullSafe neqNullSafe = 33; - BitwiseAnd bitwiseAnd = 34; - BitwiseOr bitwiseOr = 35; - BitwiseXor bitwiseXor = 36; - Remainder remainder = 37; + BinaryExpr eqNullSafe = 32; + BinaryExpr neqNullSafe = 33; + BinaryExpr bitwiseAnd = 34; + BinaryExpr bitwiseOr = 35; + BinaryExpr bitwiseXor = 36; + MathExpr remainder = 37; CaseWhen caseWhen = 38; In in = 39; - Not not = 40; + UnaryExpr not = 40; UnaryMinus unary_minus = 41; - BitwiseShiftRight bitwiseShiftRight = 42; - BitwiseShiftLeft bitwiseShiftLeft = 43; + BinaryExpr bitwiseShiftRight = 42; + BinaryExpr bitwiseShiftLeft = 43; IfExpr if = 44; NormalizeNaNAndZero normalize_nan_and_zero = 45; TruncDate truncDate = 46; TruncTimestamp truncTimestamp = 47; - BitwiseNot bitwiseNot = 48; + UnaryExpr bitwiseNot = 48; Abs abs = 49; Subquery subquery = 50; UnboundReference unbound = 51; @@ -220,35 +220,7 @@ message Literal { bool is_null = 12; } -message Add { - Expr left = 1; - Expr right = 2; - bool fail_on_error = 3; - DataType return_type = 4; -} - -message Subtract { - Expr left = 1; - Expr right = 2; - bool fail_on_error = 3; - DataType return_type = 4; -} - -message Multiply { - Expr left = 1; - Expr right = 2; - bool fail_on_error = 3; - DataType return_type = 4; -} - -message Divide { - Expr left = 1; - Expr right = 2; - bool fail_on_error = 3; - DataType return_type = 4; -} - -message Remainder { +message MathExpr { Expr left = 1; Expr right = 2; bool fail_on_error = 3; @@ -269,61 +241,12 @@ message Cast { bool allow_incompat = 5; } -message Equal { - Expr left = 1; - Expr right = 2; -} - -message NotEqual { - Expr left = 1; - Expr right = 2; -} - -message EqualNullSafe { - Expr left = 1; - Expr right = 2; -} - -message NotEqualNullSafe { - Expr left = 1; - Expr right = 2; -} - -message GreaterThan { - Expr left = 1; - Expr right = 2; -} - -message GreaterThanEqual { - Expr left = 1; - Expr right = 2; -} - -message LessThan { - Expr left = 1; - Expr right = 2; -} - -message LessThanEqual { - Expr left = 1; - Expr right = 2; -} - -message And { +message BinaryExpr { Expr left = 1; Expr right = 2; } -message Or { - Expr left = 1; - Expr right = 2; -} - -message IsNull { - Expr child = 1; -} - -message IsNotNull { +message UnaryExpr { Expr child = 1; } @@ -350,10 +273,6 @@ message Substring { int32 len = 3; } -message StringSpace { - Expr child = 1; -} - message ToJson { Expr child = 1; string timezone = 2; @@ -384,62 +303,12 @@ message CheckOverflow { bool fail_on_error = 3; } -message Like { - Expr left = 1; - Expr right = 2; -} - -message RLike { - Expr left = 1; - Expr right = 2; -} - -message StartsWith { - Expr left = 1; - Expr right = 2; -} - -message EndsWith { - Expr left = 1; - Expr right = 2; -} - -message Contains { - Expr left = 1; - Expr right = 2; -} - message ScalarFunc { string func = 1; repeated Expr args = 2; DataType return_type = 3; } -message BitwiseAnd { - Expr left = 1; - Expr right = 2; -} - -message BitwiseOr { - Expr left = 1; - Expr right = 2; -} - -message BitwiseXor { - Expr left = 1; - Expr right = 2; -} - -message BitwiseShiftRight { - Expr left = 1; - Expr right = 2; -} - -message BitwiseShiftLeft { - Expr left = 1; - Expr right = 2; -} - message CaseWhen { // The expr field is added to be consistent with CaseExpr definition in DataFusion. // This field is not really used. When constructing a CaseExpr, this expr field @@ -463,10 +332,6 @@ message NormalizeNaNAndZero { DataType datatype = 2; } -message Not { - Expr child = 1; -} - message UnaryMinus { Expr child = 1; bool fail_on_error = 2; @@ -489,10 +354,6 @@ message TruncTimestamp { string timezone = 3; } -message BitwiseNot { - Expr child = 1; -} - message Abs { Expr child = 1; EvalMode eval_mode = 2; diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 0224aabf6..0a2f7fef6 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -32,8 +32,14 @@ use arrow::{ record_batch::RecordBatch, util::display::FormatOptions, }; -use arrow_array::DictionaryArray; +use arrow_array::builder::StringBuilder; +use arrow_array::{DictionaryArray, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; +use datafusion_common::{ + cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, +}; +use datafusion_expr::ColumnarValue; +use datafusion_physical_expr::PhysicalExpr; use std::str::FromStr; use std::{ any::Any, @@ -43,12 +49,6 @@ use std::{ sync::Arc, }; -use datafusion_common::{ - cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, -}; -use datafusion_expr::ColumnarValue; -use datafusion_physical_expr::PhysicalExpr; - use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use num::{ @@ -711,6 +711,9 @@ 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)?) + } _ if is_datafusion_spark_compatible(from_type, to_type, allow_incompat) => { // use DataFusion cast only when we know that it is compatible with Spark Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) @@ -785,6 +788,7 @@ fn is_datafusion_spark_compatible( | DataType::Float64 | DataType::Decimal128(_, _) | DataType::Decimal256(_, _) + | DataType::Utf8 // note that there can be formatting differences ), DataType::Utf8 if allow_incompat => matches!( to_type, @@ -807,6 +811,52 @@ fn is_datafusion_spark_compatible( } } +fn casts_struct_to_string(array: &StructArray, timezone: &str) -> DataFusionResult { + // cast each field to a string + let string_arrays: Vec = array + .columns() + .iter() + .map(|arr| { + spark_cast( + ColumnarValue::Array(Arc::clone(arr)), + &DataType::Utf8, + EvalMode::Legacy, + timezone, + true, + ) + .and_then(|cv| cv.into_array(arr.len())) + }) + .collect::>>()?; + let string_arrays: Vec<&StringArray> = + string_arrays.iter().map(|arr| arr.as_string()).collect(); + // build the struct string containing entries in the format `"field_name":field_value` + let mut builder = StringBuilder::with_capacity(array.len(), array.len() * 16); + let mut str = String::with_capacity(array.len() * 16); + for row_index in 0..array.len() { + if array.is_null(row_index) { + builder.append_null(); + } else { + str.clear(); + let mut any_fields_written = false; + str.push('{'); + for field in &string_arrays { + if any_fields_written { + str.push_str(", "); + } + if field.is_null(row_index) { + str.push_str("null"); + } else { + str.push_str(field.value(row_index)); + } + any_fields_written = true; + } + str.push('}'); + builder.append_value(&str); + } + } + Ok(Arc::new(builder.finish())) +} + fn cast_string_to_int( to_type: &DataType, array: &ArrayRef, @@ -1879,7 +1929,7 @@ fn trim_end(s: &str) -> &str { mod tests { use arrow::datatypes::TimestampMicrosecondType; use arrow_array::StringArray; - use arrow_schema::TimeUnit; + use arrow_schema::{Field, TimeUnit}; use std::str::FromStr; use super::*; @@ -2255,4 +2305,35 @@ mod tests { ); assert!(result.is_err()) } + + #[test] + fn test_cast_struct_to_utf8() { + let a: ArrayRef = Arc::new(Int32Array::from(vec![ + Some(1), + Some(2), + None, + Some(4), + Some(5), + ])); + let b: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"])); + let c: ArrayRef = Arc::new(StructArray::from(vec![ + (Arc::new(Field::new("a", DataType::Int32, true)), a), + (Arc::new(Field::new("b", DataType::Utf8, true)), b), + ])); + let string_array = cast_array( + c, + &DataType::Utf8, + EvalMode::Legacy, + "UTC".to_owned(), + false, + ) + .unwrap(); + let string_array = string_array.as_string::(); + assert_eq!(5, string_array.len()); + assert_eq!(r#"{1, a}"#, string_array.value(0)); + assert_eq!(r#"{2, b}"#, string_array.value(1)); + assert_eq!(r#"{null, c}"#, string_array.value(2)); + assert_eq!(r#"{4, d}"#, string_array.value(3)); + assert_eq!(r#"{5, e}"#, string_array.value(4)); + } } diff --git a/native/spark-expr/src/scalar_funcs/hash_expressions.rs b/native/spark-expr/src/scalar_funcs/hash_expressions.rs index 33c8cc5e9..af423677b 100644 --- a/native/spark-expr/src/scalar_funcs/hash_expressions.rs +++ b/native/spark-expr/src/scalar_funcs/hash_expressions.rs @@ -18,7 +18,7 @@ use crate::scalar_funcs::hex::hex_strings; use crate::spark_hash::{create_murmur3_hashes, create_xxhash64_hashes}; -use arrow_array::{ArrayRef, Int32Array, Int64Array, StringArray}; +use arrow_array::{Array, ArrayRef, Int32Array, Int64Array, StringArray}; use datafusion::functions::crypto::{sha224, sha256, sha384, sha512}; use datafusion_common::cast::as_binary_array; use datafusion_common::{exec_err, internal_err, DataFusionError, ScalarValue}; @@ -139,7 +139,11 @@ fn wrap_digest_result_as_hex_string( args: &[ColumnarValue], digest: Arc, ) -> Result { - let value = digest.invoke(args)?; + let row_count = match &args[0] { + ColumnarValue::Array(array) => array.len(), + ColumnarValue::Scalar(_) => 1, + }; + let value = digest.invoke_batch(args, row_count)?; match value { ColumnarValue::Array(array) => { let binary_array = as_binary_array(&array)?; diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index c1d63299e..952ef39e9 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1275,8 +1275,14 @@ object CometSparkSessionExtensions extends Logging { .byteFromString(sparkConf.get("spark.executor.memory", "1024MB"), ByteUnit.MiB) val minimum = ConfigHelpers - .byteFromString(sparkConf.get(COMET_MEMORY_OVERHEAD_MIN_MIB.key, "384"), ByteUnit.MiB) - val overheadFactor = sparkConf.getDouble(COMET_MEMORY_OVERHEAD_FACTOR.key, 0.2) + .byteFromString( + sparkConf.get( + COMET_MEMORY_OVERHEAD_MIN_MIB.key, + COMET_MEMORY_OVERHEAD_MIN_MIB.defaultValueString), + ByteUnit.MiB) + val overheadFactor = sparkConf.getDouble( + COMET_MEMORY_OVERHEAD_FACTOR.key, + COMET_MEMORY_OVERHEAD_FACTOR.defaultValue.get) val overHeadMemFromConf = sparkConf .getOption(COMET_MEMORY_OVERHEAD.key) diff --git a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala index 9725c5f21..eb9800b8d 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -19,7 +19,7 @@ package org.apache.comet.expressions -import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType} +import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType, StructType} sealed trait SupportLevel @@ -76,7 +76,7 @@ object CometCast { case (DataTypes.StringType, _) => canCastFromString(toType, timeZoneId, evalMode) case (_, DataTypes.StringType) => - canCastToString(fromType) + canCastToString(fromType, timeZoneId, evalMode) case (DataTypes.TimestampType, _) => canCastFromTimestamp(toType) case (_: DecimalType, _) => @@ -137,7 +137,10 @@ object CometCast { } } - private def canCastToString(fromType: DataType): SupportLevel = { + private def canCastToString( + fromType: DataType, + timeZoneId: Option[String], + evalMode: CometEvalMode.Value): SupportLevel = { fromType match { case DataTypes.BooleanType => Compatible() case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | @@ -151,9 +154,26 @@ object CometCast { "There can be differences in precision. " + "For example, the input \"1.4E-45\" will produce 1.0E-45 " + "instead of 1.4E-45")) + case _: DecimalType => + // https://github.com/apache/datafusion-comet/issues/1068 + Compatible( + Some( + "There can be formatting differences in some case due to Spark using " + + "scientific notation where Comet does not")) case DataTypes.BinaryType => // https://github.com/apache/datafusion-comet/issues/377 Incompatible(Some("Only works for binary data representing valid UTF-8 strings")) + case StructType(fields) => + for (field <- fields) { + isSupported(field.dataType, DataTypes.StringType, timeZoneId, evalMode) match { + case s: Incompatible => + return s + case Unsupported => + return Unsupported + case _ => + } + } + Compatible() case _ => Unsupported } } 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 ec5302d34..b5517f40f 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -933,26 +933,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim handleCast(child, inputs, dt, timeZoneId, evalMode(c)) case add @ Add(left, right, _) if supportedDataType(left.dataType) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val addBuilder = ExprOuterClass.Add.newBuilder() - addBuilder.setLeft(leftExpr.get) - addBuilder.setRight(rightExpr.get) - addBuilder.setFailOnError(getFailOnError(add)) - serializeDataType(add.dataType).foreach { t => - addBuilder.setReturnType(t) - } - - Some( + createMathExpression(left, right, inputs, add.dataType, getFailOnError(add)).map { + expr => ExprOuterClass.Expr .newBuilder() - .setAdd(addBuilder) - .build()) - } else { - withInfo(add, left, right) - None + .setAdd(expr) + .build() } case add @ Add(left, _, _) if !supportedDataType(left.dataType) => @@ -960,26 +946,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim None case sub @ Subtract(left, right, _) if supportedDataType(left.dataType) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Subtract.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - builder.setFailOnError(getFailOnError(sub)) - serializeDataType(sub.dataType).foreach { t => - builder.setReturnType(t) - } - - Some( + createMathExpression(left, right, inputs, sub.dataType, getFailOnError(sub)).map { + expr => ExprOuterClass.Expr .newBuilder() - .setSubtract(builder) - .build()) - } else { - withInfo(sub, left, right) - None + .setSubtract(expr) + .build() } case sub @ Subtract(left, _, _) if !supportedDataType(left.dataType) => @@ -988,26 +960,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case mul @ Multiply(left, right, _) if supportedDataType(left.dataType) && !decimalBeforeSpark34(left.dataType) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Multiply.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - builder.setFailOnError(getFailOnError(mul)) - serializeDataType(mul.dataType).foreach { t => - builder.setReturnType(t) - } - - Some( + createMathExpression(left, right, inputs, mul.dataType, getFailOnError(mul)).map { + expr => ExprOuterClass.Expr .newBuilder() - .setMultiply(builder) - .build()) - } else { - withInfo(mul, left, right) - None + .setMultiply(expr) + .build() } case mul @ Multiply(left, _, _) => @@ -1021,30 +979,19 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case div @ Divide(left, right, _) if supportedDataType(left.dataType) && !decimalBeforeSpark34(left.dataType) => - val leftExpr = exprToProtoInternal(left, inputs) // Datafusion now throws an exception for dividing by zero // See https://github.com/apache/arrow-datafusion/pull/6792 // For now, use NullIf to swap zeros with nulls. - val rightExpr = exprToProtoInternal(nullIfWhenPrimitive(right), inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Divide.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - builder.setFailOnError(getFailOnError(div)) - serializeDataType(div.dataType).foreach { t => - builder.setReturnType(t) - } + val rightExpr = nullIfWhenPrimitive(right) - Some( + createMathExpression(left, rightExpr, inputs, div.dataType, getFailOnError(div)).map { + expr => ExprOuterClass.Expr .newBuilder() - .setDivide(builder) - .build()) - } else { - withInfo(div, left, right) - None + .setDivide(expr) + .build() } + case div @ Divide(left, _, _) => if (!supportedDataType(left.dataType)) { withInfo(div, s"Unsupported datatype ${left.dataType}") @@ -1056,27 +1003,16 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case rem @ Remainder(left, right, _) if supportedDataType(left.dataType) && !decimalBeforeSpark34(left.dataType) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(nullIfWhenPrimitive(right), inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Remainder.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - builder.setFailOnError(getFailOnError(rem)) - serializeDataType(rem.dataType).foreach { t => - builder.setReturnType(t) - } + val rightExpr = nullIfWhenPrimitive(right) - Some( + createMathExpression(left, rightExpr, inputs, rem.dataType, getFailOnError(rem)).map { + expr => ExprOuterClass.Expr .newBuilder() - .setRemainder(builder) - .build()) - } else { - withInfo(rem, left, right) - None + .setRemainder(expr) + .build() } + case rem @ Remainder(left, _, _) => if (!supportedDataType(left.dataType)) { withInfo(rem, s"Unsupported datatype ${left.dataType}") @@ -1087,155 +1023,67 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim None case EqualTo(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Equal.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setEq(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setEq(builder) + .build() } case Not(EqualTo(left, right)) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.NotEqual.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setNeq(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setNeq(builder) + .build() } case EqualNullSafe(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.EqualNullSafe.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setEqNullSafe(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setEqNullSafe(builder) + .build() } case Not(EqualNullSafe(left, right)) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.NotEqualNullSafe.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setNeqNullSafe(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setNeqNullSafe(builder) + .build() } case GreaterThan(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.GreaterThan.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setGt(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setGt(builder) + .build() } case GreaterThanOrEqual(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.GreaterThanEqual.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setGtEq(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setGtEq(builder) + .build() } case LessThan(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.LessThan.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setLt(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setLt(builder) + .build() } case LessThanOrEqual(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.LessThanEqual.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setLtEq(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setLtEq(builder) + .build() } case Literal(value, dataType) @@ -1373,22 +1221,11 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case Like(left, right, escapeChar) => if (escapeChar == '\\') { - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Like.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setLike(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setLike(builder) + .build() } } else { // TODO custom escape char @@ -1414,96 +1251,42 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim return None } - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.RLike.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setRlike(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setRlike(builder) + .build() } - case StartsWith(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.StartsWith.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - Some( - ExprOuterClass.Expr - .newBuilder() - .setStartsWith(builder) - .build()) - } else { - withInfo(expr, left, right) - None + case StartsWith(left, right) => + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setStartsWith(builder) + .build() } case EndsWith(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.EndsWith.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setEndsWith(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setEndsWith(builder) + .build() } case Contains(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Contains.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setContains(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setContains(builder) + .build() } case StringSpace(child) => - val childExpr = exprToProtoInternal(child, inputs) - - if (childExpr.isDefined) { - val builder = ExprOuterClass.StringSpace.newBuilder() - builder.setChild(childExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setStringSpace(builder) - .build()) - } else { - withInfo(expr, child) - None - } + createUnaryExpr( + child, + inputs, + (builder, unaryExpr) => builder.setStringSpace(unaryExpr)) case Hour(child, timeZoneId) => val childExpr = exprToProtoInternal(child, inputs) @@ -1638,38 +1421,10 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim optExprWithInfo(optExpr, expr, child) case IsNull(child) => - val childExpr = exprToProtoInternal(child, inputs) - - if (childExpr.isDefined) { - val castBuilder = ExprOuterClass.IsNull.newBuilder() - castBuilder.setChild(childExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setIsNull(castBuilder) - .build()) - } else { - withInfo(expr, child) - None - } + createUnaryExpr(child, inputs, (builder, unaryExpr) => builder.setIsNull(unaryExpr)) case IsNotNull(child) => - val childExpr = exprToProtoInternal(child, inputs) - - if (childExpr.isDefined) { - val castBuilder = ExprOuterClass.IsNotNull.newBuilder() - castBuilder.setChild(childExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setIsNotNull(castBuilder) - .build()) - } else { - withInfo(expr, child) - None - } + createUnaryExpr(child, inputs, (builder, unaryExpr) => builder.setIsNotNull(unaryExpr)) case IsNaN(child) => val childExpr = exprToProtoInternal(child, inputs) @@ -1706,41 +1461,19 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim } case And(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.And.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setAnd(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setAnd(builder) + .build() } case Or(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Or.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setOr(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setOr(builder) + .build() } case UnaryExpression(child) if expr.prettyName == "promote_precision" => @@ -2178,81 +1911,33 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim } case BitwiseAnd(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.BitwiseAnd.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setBitwiseAnd(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setBitwiseAnd(builder) + .build() } case BitwiseNot(child) => - val childExpr = exprToProtoInternal(child, inputs) - - if (childExpr.isDefined) { - val builder = ExprOuterClass.BitwiseNot.newBuilder() - builder.setChild(childExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setBitwiseNot(builder) - .build()) - } else { - withInfo(expr, child) - None - } + createUnaryExpr(child, inputs, (builder, unaryExpr) => builder.setBitwiseNot(unaryExpr)) case BitwiseOr(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.BitwiseOr.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setBitwiseOr(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setBitwiseOr(builder) + .build() } case BitwiseXor(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.BitwiseXor.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setBitwiseXor(builder) - .build()) - } else { - withInfo(expr, left, right) - None + createBinaryExpr(left, right, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setBitwiseXor(builder) + .build() } case ShiftRight(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) // DataFusion bitwise shift right expression requires // same data type between left and right side val rightExpression = if (left.dataType == LongType) { @@ -2260,25 +1945,15 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim } else { right } - val rightExpr = exprToProtoInternal(rightExpression, inputs) - - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.BitwiseShiftRight.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - Some( - ExprOuterClass.Expr - .newBuilder() - .setBitwiseShiftRight(builder) - .build()) - } else { - withInfo(expr, left, rightExpression) - None + createBinaryExpr(left, rightExpression, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setBitwiseShiftRight(builder) + .build() } case ShiftLeft(left, right) => - val leftExpr = exprToProtoInternal(left, inputs) // DataFusion bitwise shift right expression requires // same data type between left and right side val rightExpression = if (left.dataType == LongType) { @@ -2286,21 +1961,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim } else { right } - val rightExpr = exprToProtoInternal(rightExpression, inputs) - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.BitwiseShiftLeft.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setBitwiseShiftLeft(builder) - .build()) - } else { - withInfo(expr, left, rightExpression) - None + createBinaryExpr(left, rightExpression, inputs).map { builder => + ExprOuterClass.Expr + .newBuilder() + .setBitwiseShiftLeft(builder) + .build() } case In(value, list) => @@ -2319,19 +1985,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim in(expr, value, list, inputs, true) case Not(child) => - val childExpr = exprToProtoInternal(child, inputs) - if (childExpr.isDefined) { - val builder = ExprOuterClass.Not.newBuilder() - builder.setChild(childExpr.get) - Some( - ExprOuterClass.Expr - .newBuilder() - .setNot(builder) - .build()) - } else { - withInfo(expr, child) - None - } + createUnaryExpr(child, inputs, (builder, unaryExpr) => builder.setNot(unaryExpr)) case UnaryMinus(child, failOnError) => val childExpr = exprToProtoInternal(child, inputs) @@ -2612,6 +2266,88 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim } } + /** + * Creates a UnaryExpr by calling exprToProtoInternal for the provided child expression and + * then invokes the supplied function to wrap this UnaryExpr in a top-level Expr. + * + * @param child + * Spark expression + * @param inputs + * Inputs to the expression + * @param f + * Function that accepts an Expr.Builder and a UnaryExpr and builds the specific top-level + * Expr + * @return + * Some(Expr) or None if not supported + */ + def createUnaryExpr( + child: Expression, + inputs: Seq[Attribute], + f: (ExprOuterClass.Expr.Builder, ExprOuterClass.UnaryExpr) => ExprOuterClass.Expr.Builder) + : Option[ExprOuterClass.Expr] = { + val childExpr = exprToProtoInternal(child, inputs) + if (childExpr.isDefined) { + // create the generic UnaryExpr message + val inner = ExprOuterClass.UnaryExpr + .newBuilder() + .setChild(childExpr.get) + .build() + // call the user-supplied function to wrap UnaryExpr in a top-level Expr + // such as Expr.IsNull or Expr.IsNotNull + Some( + f( + ExprOuterClass.Expr + .newBuilder(), + inner).build()) + } else { + withInfo(expr, child) + None + } + } + + def createBinaryExpr( + left: Expression, + right: Expression, + inputs: Seq[Attribute]): Option[ExprOuterClass.BinaryExpr] = { + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + if (leftExpr.isDefined && rightExpr.isDefined) { + Some( + ExprOuterClass.BinaryExpr + .newBuilder() + .setLeft(leftExpr.get) + .setRight(rightExpr.get) + .build()) + } else { + withInfo(expr, left, right) + None + } + } + + def createMathExpression( + left: Expression, + right: Expression, + inputs: Seq[Attribute], + dataType: DataType, + failOnError: Boolean): Option[ExprOuterClass.MathExpr] = { + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.MathExpr.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + builder.setFailOnError(failOnError) + serializeDataType(dataType).foreach { t => + builder.setReturnType(t) + } + Some(builder.build()) + } else { + withInfo(expr, left, right) + None + } + } + def trim( expr: Expression, // parent expression srcStr: Expression, diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 4bcc3e00b..817545c5d 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -24,6 +24,7 @@ import java.io.File import scala.util.Random import scala.util.matching.Regex +import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, SaveMode} import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -470,8 +471,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateDecimalsPrecision38Scale18(), DataTypes.LongType) } - ignore("cast DecimalType(10,2) to StringType") { - // input: 0E-18, expected: 0E-18, actual: 0.000000000000000000 + test("cast DecimalType(10,2) to StringType") { castTest(generateDecimalsPrecision10Scale2(), DataTypes.StringType) } @@ -850,6 +850,37 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateTimestamps(), DataTypes.DateType) } + // Complex Types + + test("cast StructType to StringType") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "tbl") { + // primitives + checkSparkAnswerAndOperator( + "SELECT CAST(struct(_1, _2, _3, _4, _5, _6, _7, _8) as string) FROM tbl") + // TODO: enable tests for unsigned ints (_9, _10, _11, _12) once + // https://github.com/apache/datafusion-comet/issues/1067 is resolved + // checkSparkAnswerAndOperator( + // "SELECT CAST(struct(_9, _10, _11, _12) as string) FROM tbl") + // decimals + // TODO add _16 when https://github.com/apache/datafusion-comet/issues/1068 is resolved + checkSparkAnswerAndOperator("SELECT CAST(struct(_15, _17) as string) FROM tbl") + // dates & timestamps + checkSparkAnswerAndOperator("SELECT CAST(struct(_18, _19, _20) as string) FROM tbl") + // named struct + checkSparkAnswerAndOperator( + "SELECT CAST(named_struct('a', _1, 'b', _2) as string) FROM tbl") + // nested struct + checkSparkAnswerAndOperator( + "SELECT CAST(named_struct('a', named_struct('b', _1, 'c', _2)) as string) FROM tbl") + } + } + } + } + private def generateFloats(): DataFrame = { withNulls(gen.generateFloats(dataSize)).toDF("a") } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala index d787a9b1e..ad1aef4a8 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -338,7 +338,8 @@ class CometJoinSuite extends CometTestBase { } } - test("SortMergeJoin with join filter") { + // https://github.com/apache/datafusion-comet/issues/398 + ignore("SortMergeJoin with join filter") { withSQLConf( CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",